diff --git a/LICENSE b/LICENSE index 150ccc54ec6c2..1ef1f86fd704e 100644 --- a/LICENSE +++ b/LICENSE @@ -243,7 +243,7 @@ MIT License core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js core/src/main/resources/org/apache/spark/ui/static/*dataTables* core/src/main/resources/org/apache/spark/ui/static/graphlib-dot.min.js -ore/src/main/resources/org/apache/spark/ui/static/jquery* +core/src/main/resources/org/apache/spark/ui/static/jquery* core/src/main/resources/org/apache/spark/ui/static/sorttable.js docs/js/vendor/anchor.min.js docs/js/vendor/jquery* diff --git a/LICENSE-binary b/LICENSE-binary index 16da39145d050..7865d9df63144 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -218,13 +218,14 @@ javax.jdo:jdo-api joda-time:joda-time net.sf.opencsv:opencsv org.apache.derby:derby +org.ehcache:ehcache org.objenesis:objenesis org.roaringbitmap:RoaringBitmap 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 +org.apache.xbean:xbean-asm7-shaded com.squareup.okhttp3:logging-interceptor com.squareup.okhttp3:okhttp com.squareup.okio:okio @@ -253,12 +254,14 @@ commons-codec:commons-codec commons-collections:commons-collections io.fabric8:kubernetes-client io.fabric8:kubernetes-model +io.fabric8:kubernetes-model-common io.netty:netty-all net.hydromatic:eigenbase-properties net.sf.supercsv:super-csv org.apache.arrow:arrow-format org.apache.arrow:arrow-memory org.apache.arrow:arrow-vector +org.apache.commons:commons-configuration2 org.apache.commons:commons-crypto org.apache.commons:commons-lang3 org.apache.hadoop:hadoop-annotations @@ -266,6 +269,7 @@ org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-client org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-hdfs +org.apache.hadoop:hadoop-hdfs-client org.apache.hadoop:hadoop-mapreduce-client-app org.apache.hadoop:hadoop-mapreduce-client-common org.apache.hadoop:hadoop-mapreduce-client-core @@ -278,6 +282,21 @@ org.apache.hadoop:hadoop-yarn-server-common org.apache.hadoop:hadoop-yarn-server-web-proxy org.apache.httpcomponents:httpclient org.apache.httpcomponents:httpcore +org.apache.kerby:kerb-admin +org.apache.kerby:kerb-client +org.apache.kerby:kerb-common +org.apache.kerby:kerb-core +org.apache.kerby:kerb-crypto +org.apache.kerby:kerb-identity +org.apache.kerby:kerb-server +org.apache.kerby:kerb-simplekdc +org.apache.kerby:kerb-util +org.apache.kerby:kerby-asn1 +org.apache.kerby:kerby-config +org.apache.kerby:kerby-pkix +org.apache.kerby:kerby-util +org.apache.kerby:kerby-xdr +org.apache.kerby:token-provider org.apache.orc:orc-core org.apache.orc:orc-mapreduce org.mortbay.jetty:jetty @@ -292,16 +311,24 @@ com.fasterxml.jackson.core:jackson-annotations com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.dataformat:jackson-dataformat-yaml +com.fasterxml.jackson.jaxrs:jackson-jaxrs-base +com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider com.fasterxml.jackson.module:jackson-module-jaxb-annotations com.fasterxml.jackson.module:jackson-module-paranamer com.fasterxml.jackson.module:jackson-module-scala_2.12 +com.fasterxml.woodstox:woodstox-core com.github.mifmif:generex +com.github.stephenc.jcip:jcip-annotations com.google.code.findbugs:jsr305 com.google.code.gson:gson +com.google.flatbuffers:flatbuffers-java +com.google.guava:guava com.google.inject:guice com.google.inject.extensions:guice-servlet +com.nimbusds:nimbus-jose-jwt com.twitter:parquet-hadoop-bundle commons-cli:commons-cli +commons-daemon:commons-daemon commons-dbcp:commons-dbcp commons-io:commons-io commons-lang:commons-lang @@ -313,6 +340,8 @@ javax.inject:javax.inject javax.validation:validation-api log4j:apache-log4j-extras log4j:log4j +net.minidev:accessors-smart +net.minidev:json-smart net.sf.jpam:jpam org.apache.avro:avro org.apache.avro:avro-ipc @@ -328,6 +357,7 @@ org.apache.directory.server:apacheds-i18n org.apache.directory.server:apacheds-kerberos-codec org.apache.htrace:htrace-core org.apache.ivy:ivy +org.apache.geronimo.specs:geronimo-jcache_1.0_spec org.apache.mesos:mesos org.apache.parquet:parquet-column org.apache.parquet:parquet-common @@ -343,11 +373,6 @@ org.datanucleus:datanucleus-api-jdo org.datanucleus:datanucleus-core org.datanucleus:datanucleus-rdbms org.lz4:lz4-java -org.spark-project.hive:hive-beeline -org.spark-project.hive:hive-cli -org.spark-project.hive:hive-exec -org.spark-project.hive:hive-jdbc -org.spark-project.hive:hive-metastore org.xerial.snappy:snappy-java stax:stax-api xerces:xercesImpl @@ -369,6 +394,27 @@ org.eclipse.jetty:jetty-webapp org.eclipse.jetty:jetty-xml org.scala-lang.modules:scala-xml_2.12 org.opencypher:okapi-shade +com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter +com.zaxxer.HikariCP +org.apache.hive:hive-beeline +org.apache.hive:hive-cli +org.apache.hive:hive-common +org.apache.hive:hive-exec +org.apache.hive:hive-jdbc +org.apache.hive:hive-llap-common +org.apache.hive:hive-metastore +org.apache.hive:hive-serde +org.apache.hive:hive-service-rpc +org.apache.hive:hive-shims-0.23 +org.apache.hive:hive-shims +org.apache.hive:hive-common +org.apache.hive:hive-shims-scheduler +org.apache.hive:hive-storage-api +org.apache.hive:hive-vector-code-gen +org.datanucleus:javax.jdo +com.tdunning:json +org.apache.velocity:velocity +org.apache.yetus:audience-annotations core/src/main/java/org/apache/spark/util/collection/TimSort.java core/src/main/resources/org/apache/spark/ui/static/bootstrap* @@ -387,6 +433,7 @@ BSD 2-Clause ------------ com.github.luben:zstd-jni +dnsjava:dnsjava javolution:javolution com.esotericsoftware:kryo-shaded com.esotericsoftware:minlog @@ -394,6 +441,7 @@ com.esotericsoftware:reflectasm com.google.protobuf:protobuf-java org.codehaus.janino:commons-compiler org.codehaus.janino:janino +org.codehaus.woodstox:stax2-api jline:jline org.jodd:jodd-core com.github.wendykierp:JTransforms @@ -410,6 +458,7 @@ org.antlr:stringtemplate org.antlr:antlr4-runtime antlr:antlr com.github.fommil.netlib:core +com.google.re2j:re2j com.thoughtworks.paranamer:paranamer org.scala-lang:scala-compiler org.scala-lang:scala-library @@ -435,6 +484,7 @@ is distributed under the 3-Clause BSD license. MIT License ----------- +com.microsoft.sqlserver:mssql-jdbc org.typelevel:spire_2.12 org.typelevel:spire-macros_2.12 org.typelevel:spire-platform_2.12 @@ -464,6 +514,7 @@ Common Development and Distribution License (CDDL) 1.0 javax.activation:activation http://www.oracle.com/technetwork/java/javase/tech/index-jsp-138795.html javax.xml.stream:stax-api https://jcp.org/en/jsr/detail?id=173 +javax.transaction:javax.transaction-api Common Development and Distribution License (CDDL) 1.1 @@ -471,6 +522,7 @@ Common Development and Distribution License (CDDL) 1.1 javax.el:javax.el-api https://javaee.github.io/uel-ri/ javax.servlet:javax.servlet-api https://javaee.github.io/servlet-spec/ +javax.servlet.jsp:jsp-api javax.transaction:jta http://www.oracle.com/technetwork/java/index.html javax.xml.bind:jaxb-api https://github.com/javaee/jaxb-v2 org.glassfish.hk2:hk2-api https://github.com/javaee/glassfish @@ -492,6 +544,7 @@ Eclipse Distribution License (EDL) 1.0 -------------------------------------- org.glassfish.jaxb:jaxb-runtime +jakarta.activation:jakarta.activation-api jakarta.xml.bind:jakarta.xml.bind-api com.sun.istack:istack-commons-runtime @@ -501,6 +554,7 @@ Eclipse Public License (EPL) 2.0 jakarta.annotation:jakarta-annotation-api https://projects.eclipse.org/projects/ee4j.ca jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api +org.glassfish.hk2.external:jakarta.inject Python Software Foundation License diff --git a/NOTICE-binary b/NOTICE-binary index f93e088a9a731..d99c2d1c64c22 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -65,8 +65,8 @@ Copyright 2009-2014 The Apache Software Foundation Objenesis Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita -Apache XBean :: ASM 5 shaded (repackaged) -Copyright 2005-2015 The Apache Software Foundation +Apache XBean :: ASM shaded (repackaged) +Copyright 2005-2019 The Apache Software Foundation -------------------------------------- @@ -661,6 +661,9 @@ Copyright 2017 The Apache Software Foundation Apache Commons CLI Copyright 2001-2009 The Apache Software Foundation +Apache Commons Daemon +Copyright 1999-2019 The Apache Software Foundation + Google Guice - Extensions - Servlet Copyright 2006-2011 Google, Inc. @@ -1135,4 +1138,380 @@ Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 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 +limitations under the License. + +dropwizard-metrics-hadoop-metrics2-reporter +Copyright 2016 Josh Elser + +Hive Beeline +Copyright 2019 The Apache Software Foundation + +Hive CLI +Copyright 2019 The Apache Software Foundation + +Hive Common +Copyright 2019 The Apache Software Foundation + +Hive JDBC +Copyright 2019 The Apache Software Foundation + +Hive Query Language +Copyright 2019 The Apache Software Foundation + +Hive Llap Common +Copyright 2019 The Apache Software Foundation + +Hive Metastore +Copyright 2019 The Apache Software Foundation + +Hive Serde +Copyright 2019 The Apache Software Foundation + +Hive Service RPC +Copyright 2019 The Apache Software Foundation + +Hive Shims +Copyright 2019 The Apache Software Foundation + +Hive Shims 0.23 +Copyright 2019 The Apache Software Foundation + +Hive Shims Common +Copyright 2019 The Apache Software Foundation + +Hive Shims Scheduler +Copyright 2019 The Apache Software Foundation + +Hive Storage API +Copyright 2018 The Apache Software Foundation + +Hive Vector-Code-Gen Utilities +Copyright 2019 The Apache Software Foundation + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + Copyright 2015-2015 DataNucleus + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +Android JSON library +Copyright (C) 2010 The Android Open Source Project + +This product includes software developed by +The Android Open Source Project + +Apache Velocity + +Copyright (C) 2000-2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Apache Yetus - Audience Annotations +Copyright 2015-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Ehcache V3 +Copyright 2014-2016 Terracotta, Inc. + +The product includes software from the Apache Commons Lang project, +under the Apache License 2.0 (see: org.ehcache.impl.internal.classes.commonslang) + +Apache Geronimo JCache Spec 1.0 +Copyright 2003-2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Admin +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Client +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Common +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb core +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Crypto +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Identity +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Server +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerb Simple Kdc +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby-kerb Util +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby ASN1 Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby Config +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby PKIX Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby Util +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Kerby XDR Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Token provider +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/R/check-cran.sh b/R/check-cran.sh index 22cc9c6b601fc..22c8f423cfd12 100755 --- a/R/check-cran.sh +++ b/R/check-cran.sh @@ -65,6 +65,10 @@ fi echo "Running CRAN check with $CRAN_CHECK_OPTIONS options" +# Remove this environment variable to allow to check suggested packages once +# Jenkins installs arrow. See SPARK-29339. +export _R_CHECK_FORCE_SUGGESTS_=FALSE + if [ -n "$NO_TESTS" ] && [ -n "$NO_MANUAL" ] then "$R_SCRIPT_PATH/R" CMD check $CRAN_CHECK_OPTIONS "SparkR_$VERSION.tar.gz" diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f4780862099d3..95d3e52bef3a9 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -22,7 +22,8 @@ Suggests: rmarkdown, testthat, e1071, - survival + survival, + arrow Collate: 'schema.R' 'generics.R' diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 43ea27b359a9c..f27ef4ee28f16 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -148,19 +148,7 @@ getDefaultSqlSource <- function() { } 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) - + if (requireNamespace("arrow", quietly = TRUE)) { numPartitions <- if (!is.null(numPartitions)) { numToInt(numPartitions) } else { @@ -176,11 +164,11 @@ writeToFileInArrow <- function(fileName, rdf, numPartitions) { stream_writer <- NULL tryCatch({ for (rdf_slice in rdf_slices) { - batch <- record_batch(rdf_slice) + batch <- arrow::record_batch(rdf_slice) if (is.null(stream_writer)) { - stream <- FileOutputStream(fileName) + stream <- arrow::FileOutputStream(fileName) schema <- batch$schema - stream_writer <- RecordBatchStreamWriter(stream, schema) + stream_writer <- arrow::RecordBatchStreamWriter(stream, schema) } stream_writer$write_batch(batch) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index b38d245a0cca7..a6febb1cbd132 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -232,11 +232,7 @@ readMultipleObjectsWithKeys <- function(inputCon) { } 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) + if (requireNamespace("arrow", quietly = TRUE)) { # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. useAsTibble <- exists("as_tibble", envir = asNamespace("arrow")) @@ -246,7 +242,7 @@ readDeserializeInArrow <- function(inputCon) { # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") - batches <- RecordBatchStreamReader(arrowData)$batches() + batches <- arrow::RecordBatchStreamReader(arrowData)$batches() if (useAsTibble) { as_tibble <- get("as_tibble", envir = asNamespace("arrow")) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index eecb84572a30b..eec221c2be4bf 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -3617,7 +3617,7 @@ setMethod("size", #' @details #' \code{slice}: Returns an array containing all the elements in x from the index start -#' (or starting from the end if start is negative) with the specified length. +#' (array indices start at 1, or from the end if start is negative) with the specified length. #' #' @rdname column_collection_functions #' @param start an index indicating the first element occurring in the result. diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 0d6f32c8f7e1f..cb3c1c59d12ed 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -222,15 +222,11 @@ 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) - + if (requireNamespace("arrow", quietly = TRUE)) { # 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())) + writeRaw(conn, arrow::write_arrow(df, raw())) } else { stop("'arrow' package should be installed.") } diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 80dc4ee634512..dfe69b7f4f1fb 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -50,7 +50,7 @@ compute <- function(mode, partition, serializer, deserializer, key, } else { # Check to see if inputData is a valid data.frame stopifnot(deserializer == "byte" || deserializer == "arrow") - stopifnot(class(inputData) == "data.frame") + stopifnot(is.data.frame(inputData)) } if (mode == 2) { diff --git a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R index 825c7423e1579..97972753a78fa 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R +++ b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R @@ -101,7 +101,7 @@ test_that("dapply() Arrow optimization", { tryCatch({ ret <- dapply(df, function(rdf) { - stopifnot(class(rdf) == "data.frame") + stopifnot(is.data.frame(rdf)) rdf }, schema(df)) @@ -115,7 +115,7 @@ test_that("dapply() Arrow optimization", { tryCatch({ ret <- dapply(df, function(rdf) { - stopifnot(class(rdf) == "data.frame") + stopifnot(is.data.frame(rdf)) # mtcars' hp is more then 50. stopifnot(all(rdf$hp > 50)) rdf @@ -199,7 +199,7 @@ test_that("gapply() Arrow optimization", { if (length(key) > 0) { stopifnot(is.numeric(key[[1]])) } - stopifnot(class(grouped) == "data.frame") + stopifnot(is.data.frame(grouped)) grouped }, schema(df)) @@ -217,7 +217,7 @@ test_that("gapply() Arrow optimization", { if (length(key) > 0) { stopifnot(is.numeric(key[[1]])) } - stopifnot(class(grouped) == "data.frame") + stopifnot(is.data.frame(grouped)) stopifnot(length(colnames(grouped)) == 11) # mtcars' hp is more then 50. stopifnot(all(grouped$hp > 50)) diff --git a/appveyor.yml b/appveyor.yml index a61436c5d2e68..b36175a787ae9 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -43,12 +43,15 @@ install: - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + # Use Arrow R 0.14.1 for now. 0.15.0 seems not working for now. See SPARK-29378. + - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" + - cmd: R -e "install.packages('https://cran.r-project.org/src/contrib/Archive/arrow/arrow_0.14.1.tar.gz', repos=NULL, type='source')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # 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')" + - cmd: R -e "packageVersion('knitr'); packageVersion('rmarkdown'); packageVersion('testthat'); packageVersion('e1071'); packageVersion('survival'); packageVersion('arrow')" build_script: # '-Djna.nosys=true' is required to avoid kernel32.dll load failure. 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 736059fdd1f57..490915f6de4b3 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 @@ -112,4 +112,27 @@ public static int[] decode(ByteBuf buf) { return ints; } } + + /** Long integer arrays are encoded with their length followed by long integers. */ + public static class LongArrays { + public static int encodedLength(long[] longs) { + return 4 + 8 * longs.length; + } + + public static void encode(ByteBuf buf, long[] longs) { + buf.writeInt(longs.length); + for (long i : longs) { + buf.writeLong(i); + } + } + + public static long[] decode(ByteBuf buf) { + int numLongs = buf.readInt(); + long[] longs = new long[numLongs]; + for (int i = 0; i < longs.length; i ++) { + longs[i] = buf.readLong(); + } + return longs; + } + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 037e5cf7e5222..b886fce9be21a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -102,11 +102,15 @@ protected void handleMessage( FetchShuffleBlocks msg = (FetchShuffleBlocks) msgObj; checkAuth(client, msg.appId); numBlockIds = 0; - for (int[] ids: msg.reduceIds) { - numBlockIds += ids.length; + if (msg.batchFetchEnabled) { + numBlockIds = msg.mapIds.length; + } else { + for (int[] ids: msg.reduceIds) { + numBlockIds += ids.length; + } } streamId = streamManager.registerStream(client.getClientId(), - new ManagedBufferIterator(msg, numBlockIds), client.getChannel()); + new ShuffleManagedBufferIterator(msg), client.getChannel()); } else { // For the compatibility with the old version, still keep the support for OpenBlocks. OpenBlocks msg = (OpenBlocks) msgObj; @@ -299,21 +303,6 @@ private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { 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 < size; @@ -328,6 +317,59 @@ public ManagedBuffer next() { } } + private class ShuffleManagedBufferIterator implements Iterator { + + private int mapIdx = 0; + private int reduceIdx = 0; + + private final String appId; + private final String execId; + private final int shuffleId; + private final long[] mapIds; + private final int[][] reduceIds; + private final boolean batchFetchEnabled; + + ShuffleManagedBufferIterator(FetchShuffleBlocks msg) { + appId = msg.appId; + execId = msg.execId; + shuffleId = msg.shuffleId; + mapIds = msg.mapIds; + reduceIds = msg.reduceIds; + batchFetchEnabled = msg.batchFetchEnabled; + } + + @Override + public boolean hasNext() { + // mapIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks + // must have non-empty mapIds and reduceIds, see the checking logic in + // OneForOneBlockFetcher. + assert(mapIds.length != 0 && mapIds.length == reduceIds.length); + return mapIdx < mapIds.length && reduceIdx < reduceIds[mapIdx].length; + } + + @Override + public ManagedBuffer next() { + ManagedBuffer block; + if (!batchFetchEnabled) { + block = blockManager.getBlockData( + appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); + if (reduceIdx < reduceIds[mapIdx].length - 1) { + reduceIdx += 1; + } else { + reduceIdx = 0; + mapIdx += 1; + } + } else { + assert(reduceIds[mapIdx].length == 2); + block = blockManager.getContinuousBlocksData(appId, execId, shuffleId, mapIds[mapIdx], + reduceIds[mapIdx][0], reduceIds[mapIdx][1]); + mapIdx += 1; + } + metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); + return block; + } + } + @Override public void channelActive(TransportClient client) { metrics.activeConnections.inc(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index b8e52c8621fb6..85d278138c2b4 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -53,7 +53,7 @@ public class ExternalBlockStoreClient extends BlockStoreClient { private final SecretKeyHolder secretKeyHolder; private final long registrationTimeoutMs; - protected TransportClientFactory clientFactory; + protected volatile TransportClientFactory clientFactory; protected String appId; /** @@ -102,9 +102,14 @@ public void fetchBlocks( try { RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = (blockIds1, listener1) -> { - TransportClient client = clientFactory.createClient(host, port); - new OneForOneBlockFetcher(client, appId, execId, - blockIds1, listener1, conf, downloadFileManager).start(); + // Unless this client is closed. + if (clientFactory != null) { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockFetcher(client, appId, execId, + blockIds1, listener1, conf, downloadFileManager).start(); + } else { + logger.info("This clientFactory was closed. Skipping further block fetch retries."); + } }; int maxRetries = conf.maxIORetries(); 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 50f16fc700f12..beca5d6e5a788 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 @@ -165,21 +165,34 @@ public void registerExecutor( } /** - * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions - * about how the hash and sort based shuffles store their data. + * Obtains a FileSegmentManagedBuffer from a single block (shuffleId, mapId, reduceId). */ public ManagedBuffer getBlockData( String appId, String execId, int shuffleId, - int mapId, + long mapId, int reduceId) { + return getContinuousBlocksData(appId, execId, shuffleId, mapId, reduceId, reduceId + 1); + } + + /** + * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, [startReduceId, endReduceId)). + * We make assumptions about how the hash and sort based shuffles store their data. + */ + public ManagedBuffer getContinuousBlocksData( + String appId, + String execId, + int shuffleId, + long mapId, + int startReduceId, + int endReduceId) { 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 getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); + return getSortBasedShuffleBlockData(executor, shuffleId, mapId, startReduceId, endReduceId); } public ManagedBuffer getRddBlockData( @@ -296,13 +309,14 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, long mapId, int startReduceId, int endReduceId) { File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.index"); try { ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile); - ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(reduceId); + ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex( + startReduceId, endReduceId); return new FileSegmentManagedBuffer( conf, ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, 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 cc11e92067375..ab373a7f03d9d 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 @@ -24,6 +24,7 @@ import java.util.HashMap; import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,40 +112,47 @@ private boolean isShuffleBlocks(String[] blockIds) { */ private FetchShuffleBlocks createFetchShuffleBlocksMsg( String appId, String execId, String[] blockIds) { - int shuffleId = splitBlockId(blockIds[0])[0]; - HashMap> mapIdToReduceIds = new HashMap<>(); + String[] firstBlock = splitBlockId(blockIds[0]); + int shuffleId = Integer.parseInt(firstBlock[1]); + boolean batchFetchEnabled = firstBlock.length == 5; + + HashMap> mapIdToReduceIds = new HashMap<>(); for (String blockId : blockIds) { - int[] blockIdParts = splitBlockId(blockId); - if (blockIdParts[0] != shuffleId) { + String[] blockIdParts = splitBlockId(blockId); + if (Integer.parseInt(blockIdParts[1]) != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); } - int mapId = blockIdParts[1]; + long mapId = Long.parseLong(blockIdParts[2]); if (!mapIdToReduceIds.containsKey(mapId)) { mapIdToReduceIds.put(mapId, new ArrayList<>()); } - mapIdToReduceIds.get(mapId).add(blockIdParts[2]); + mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[3])); + if (batchFetchEnabled) { + // When we read continuous shuffle blocks in batch, we will reuse reduceIds in + // FetchShuffleBlocks to store the start and end reduce id for range + // [startReduceId, endReduceId). + assert(blockIdParts.length == 5); + mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[4])); + } } - int[] mapIds = Ints.toArray(mapIdToReduceIds.keySet()); + long[] mapIds = Longs.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); + return new FetchShuffleBlocks( + appId, execId, shuffleId, mapIds, reduceIdArr, batchFetchEnabled); } - /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ - private int[] splitBlockId(String blockId) { + /** Split the shuffleBlockId and return shuffleId, mapId and reduceIds. */ + private String[] splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { + if (blockIdParts.length < 4 || blockIdParts.length > 5 || !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]) - }; + return blockIdParts; } /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java index 371149bef3974..b65aacfcc4b9e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java @@ -54,8 +54,15 @@ public int getSize() { * Get index offset for a particular reducer. */ public ShuffleIndexRecord getIndex(int reduceId) { - long offset = offsets.get(reduceId); - long nextOffset = offsets.get(reduceId + 1); + return getIndex(reduceId, reduceId + 1); + } + + /** + * Get index offset for the reducer range of [startReduceId, endReduceId). + */ + public ShuffleIndexRecord getIndex(int startReduceId, int endReduceId) { + long offset = offsets.get(startReduceId); + long nextOffset = offsets.get(endReduceId); return new ShuffleIndexRecord(offset, nextOffset - offset); } } 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 index 466eeb3e048a8..c0f307af042e2 100644 --- 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 @@ -34,21 +34,33 @@ public class FetchShuffleBlocks extends BlockTransferMessage { 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 long[] mapIds; + // When batchFetchEnabled=true, reduceIds[i] contains 2 elements: startReduceId (inclusive) and + // endReduceId (exclusive) for the mapper mapIds[i]. + // When batchFetchEnabled=false, reduceIds[i] contains all the reduce IDs that mapper mapIds[i] + // needs to fetch. public final int[][] reduceIds; + public final boolean batchFetchEnabled; public FetchShuffleBlocks( String appId, String execId, int shuffleId, - int[] mapIds, - int[][] reduceIds) { + long[] mapIds, + int[][] reduceIds, + boolean batchFetchEnabled) { this.appId = appId; this.execId = execId; this.shuffleId = shuffleId; this.mapIds = mapIds; this.reduceIds = reduceIds; assert(mapIds.length == reduceIds.length); + this.batchFetchEnabled = batchFetchEnabled; + if (batchFetchEnabled) { + for (int[] ids: reduceIds) { + assert(ids.length == 2); + } + } } @Override @@ -62,6 +74,7 @@ public String toString() { .add("shuffleId", shuffleId) .add("mapIds", Arrays.toString(mapIds)) .add("reduceIds", Arrays.deepToString(reduceIds)) + .add("batchFetchEnabled", batchFetchEnabled) .toString(); } @@ -73,6 +86,7 @@ public boolean equals(Object o) { FetchShuffleBlocks that = (FetchShuffleBlocks) o; if (shuffleId != that.shuffleId) return false; + if (batchFetchEnabled != that.batchFetchEnabled) return false; if (!appId.equals(that.appId)) return false; if (!execId.equals(that.execId)) return false; if (!Arrays.equals(mapIds, that.mapIds)) return false; @@ -86,6 +100,7 @@ public int hashCode() { result = 31 * result + shuffleId; result = 31 * result + Arrays.hashCode(mapIds); result = 31 * result + Arrays.deepHashCode(reduceIds); + result = 31 * result + (batchFetchEnabled ? 1 : 0); return result; } @@ -98,9 +113,10 @@ public int encodedLength() { return Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(execId) + 4 /* encoded length of shuffleId */ - + Encoders.IntArrays.encodedLength(mapIds) + + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ - + encodedLengthOfReduceIds; + + encodedLengthOfReduceIds + + 1; /* encoded length of batchFetchEnabled */ } @Override @@ -108,23 +124,25 @@ public void encode(ByteBuf buf) { Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, execId); buf.writeInt(shuffleId); - Encoders.IntArrays.encode(buf, mapIds); + Encoders.LongArrays.encode(buf, mapIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { Encoders.IntArrays.encode(buf, ids); } + buf.writeBoolean(batchFetchEnabled); } 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); + long[] mapIds = Encoders.LongArrays.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); + boolean batchFetchEnabled = buf.readBoolean(); + return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIds, batchFetchEnabled); } } 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 649c471dc1679..fd2c67a3a270a 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 @@ -29,8 +29,11 @@ public class BlockTransferMessagesSuite { 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 }})); + "app-1", "exec-2", 0, new long[] {0, 1}, + new int[][] {{ 0, 1 }, { 0, 1, 2 }}, false)); + checkSerializeDeserialize(new FetchShuffleBlocks( + "app-1", "exec-2", 0, new long[] {0, 1}, + new int[][] {{ 0, 1 }, { 0, 2 }}, true)); 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/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 9c623a70424b6..455351fcf767c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -101,7 +101,7 @@ public void testFetchShuffleBlocks() { 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 }}); + "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}, false); checkOpenBlocksReceive(fetchShuffleBlocks, blockMarkers); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); @@ -109,6 +109,22 @@ public void testFetchShuffleBlocks() { verifyOpenBlockLatencyMetrics(); } + @Test + public void testFetchShuffleBlocksInBatch() { + ManagedBuffer[] batchBlockMarkers = { + new NioManagedBuffer(ByteBuffer.wrap(new byte[10])) + }; + when(blockResolver.getContinuousBlocksData( + "app0", "exec1", 0, 0, 0, 1)).thenReturn(batchBlockMarkers[0]); + + FetchShuffleBlocks fetchShuffleBlocks = new FetchShuffleBlocks( + "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}, true); + checkOpenBlocksReceive(fetchShuffleBlocks, batchBlockMarkers); + + verify(blockResolver, times(1)).getContinuousBlocksData("app0", "exec1", 0, 0, 0, 1); + verifyOpenBlockLatencyMetrics(); + } + @Test public void testOpenDiskPersistedRDDBlocks() { when(blockResolver.getRddBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); @@ -154,7 +170,7 @@ private void checkOpenBlocksReceive(BlockTransferMessage msg, ManagedBuffer[] bl StreamHandle handle = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); - assertEquals(2, handle.numChunks); + assertEquals(blockMarkers.length, handle.numChunks); @SuppressWarnings("unchecked") ArgumentCaptor> stream = (ArgumentCaptor>) @@ -162,8 +178,9 @@ private void checkOpenBlocksReceive(BlockTransferMessage msg, ManagedBuffer[] bl verify(streamManager, times(1)).registerStream(anyString(), stream.capture(), any()); Iterator buffers = stream.getValue(); - assertEquals(blockMarkers[0], buffers.next()); - assertEquals(blockMarkers[1], buffers.next()); + for (ManagedBuffer blockMarker : blockMarkers) { + assertEquals(blockMarker, buffers.next()); + } assertFalse(buffers.hasNext()); } 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 09eb699be305a..09b31430b1eb9 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 @@ -111,6 +111,13 @@ public void testSortShuffleBlocks() throws IOException { CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); assertEquals(sortBlock1, block1); } + + try (InputStream blocksStream = resolver.getContinuousBlocksData( + "app0", "exec0", 0, 0, 0, 2).createInputStream()) { + String blocks = + CharStreams.toString(new InputStreamReader(blocksStream, StandardCharsets.UTF_8)); + assertEquals(sortBlock0 + sortBlock1, blocks); + } } @Test 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 66633cc7a3595..285eedb39c65c 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 @@ -64,7 +64,7 @@ public void testFetchOne() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0 }}), + new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0 }}, false), conf); verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); @@ -100,7 +100,8 @@ public void testFetchThreeShuffleBlocks() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0, 1, 2 }}), + new FetchShuffleBlocks( + "app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 1, 2 }}, false), conf); for (int i = 0; i < 3; i ++) { @@ -109,6 +110,23 @@ public void testFetchThreeShuffleBlocks() { } } + @Test + public void testBatchFetchThreeShuffleBlocks() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[58]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks( + "app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 3 }}, true), + conf); + + verify(listener, times(1)).onBlockFetchSuccess( + "shuffle_0_0_0_3", blocks.get("shuffle_0_0_0_3")); + } + @Test public void testFetchThree() { LinkedHashMap blocks = Maps.newLinkedHashMap(); 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 908ff1983e6be..184ddac9a71a6 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,7 +18,6 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; -import java.util.Locale; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -33,84 +32,11 @@ public final class CalendarInterval implements Serializable { public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24; public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; - /** - * A function to generate regex which matches interval string's unit part like "3 years". - * - * First, we can leave out some units in interval string, and we only care about the value of - * unit, so here we use non-capturing group to wrap the actual regex. - * At the beginning of the actual regex, we should match spaces before the unit part. - * Next is the number part, starts with an optional "-" to represent negative value. We use - * capturing group to wrap this part as we need the value later. - * Finally is the unit name, ends with an optional "s". - */ - private static String unitRegex(String unit) { - return "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?"; - } - - private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + - unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + - unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond")); - - private static Pattern yearMonthPattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); + private static Pattern yearMonthPattern = Pattern.compile( + "^([+|-])?(\\d+)-(\\d+)$"); private static Pattern dayTimePattern = Pattern.compile( - "^(?:['|\"])?([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); - - private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); - - private static long toLong(String s) { - if (s == null) { - return 0; - } else { - return Long.parseLong(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; - } - s = s.trim(); - Matcher m = p.matcher(s); - if (!m.matches() || s.equals("interval")) { - return null; - } else { - long months = toLong(m.group(1)) * 12 + toLong(m.group(2)); - long microseconds = toLong(m.group(3)) * MICROS_PER_WEEK; - microseconds += toLong(m.group(4)) * MICROS_PER_DAY; - microseconds += toLong(m.group(5)) * MICROS_PER_HOUR; - microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE; - microseconds += toLong(m.group(7)) * MICROS_PER_SECOND; - microseconds += toLong(m.group(8)) * MICROS_PER_MILLI; - microseconds += toLong(m.group(9)); - return new CalendarInterval((int) months, microseconds); - } - } - - /** - * 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; - } + "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$"); public static long toLongWithRange(String fieldName, String s, long minValue, long maxValue) throws IllegalArgumentException { @@ -233,72 +159,59 @@ public static CalendarInterval fromDayTimeString(String s, String from, String t return result; } - public static CalendarInterval fromSingleUnitString(String unit, String s) + public static CalendarInterval fromUnitStrings(String[] units, String[] values) throws IllegalArgumentException { + assert units.length == values.length; + int months = 0; + long microseconds = 0; - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException(String.format("Interval %s string was null", unit)); - } - s = s.trim(); - Matcher m = quoteTrimPattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); - } else { + for (int i = 0; i < units.length; i++) { try { - switch (unit) { + switch (units[i]) { case "year": - int year = (int) toLongWithRange("year", m.group(1), - Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); - result = new CalendarInterval(year * 12, 0L); + months = Math.addExact(months, Math.multiplyExact(Integer.parseInt(values[i]), 12)); break; case "month": - int month = (int) toLongWithRange("month", m.group(1), - Integer.MIN_VALUE, Integer.MAX_VALUE); - result = new CalendarInterval(month, 0L); + months = Math.addExact(months, Integer.parseInt(values[i])); break; case "week": - long week = toLongWithRange("week", m.group(1), - Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); - result = new CalendarInterval(0, week * MICROS_PER_WEEK); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_WEEK)); break; case "day": - long day = toLongWithRange("day", m.group(1), - Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); - result = new CalendarInterval(0, day * MICROS_PER_DAY); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_DAY)); break; case "hour": - long hour = toLongWithRange("hour", m.group(1), - Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); - result = new CalendarInterval(0, hour * MICROS_PER_HOUR); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_HOUR)); break; case "minute": - long minute = toLongWithRange("minute", m.group(1), - Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); - result = new CalendarInterval(0, minute * MICROS_PER_MINUTE); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MINUTE)); break; case "second": { - long micros = parseSecondNano(m.group(1)); - result = new CalendarInterval(0, micros); + microseconds = Math.addExact(microseconds, parseSecondNano(values[i])); break; } case "millisecond": - long millisecond = toLongWithRange("millisecond", m.group(1), - Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); - result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MILLI)); break; - case "microsecond": { - long micros = Long.parseLong(m.group(1)); - result = new CalendarInterval(0, micros); + case "microsecond": + microseconds = Math.addExact(microseconds, Long.parseLong(values[i])); break; - } } } catch (Exception e) { throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e); } } - return result; + return new CalendarInterval(months, microseconds); } /** 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 c307d74e0ba07..9f3262bf2aaa4 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 @@ -60,75 +60,6 @@ public void toStringTest() { assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); } - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - String input; - - input = "interval -5 years 23 month"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); - - input = "interval -5 years 23 month "; - assertEquals(fromString(input), result); - - input = " interval -5 years 23 month "; - assertEquals(fromString(input), result); - - // Error cases - input = "interval 3month 1 hour"; - assertNull(fromString(input)); - - input = "interval 3 moth 1 hour"; - assertNull(fromString(input)); - - input = "interval"; - assertNull(fromString(input)); - - input = "int"; - assertNull(fromString(input)); - - input = ""; - assertNull(fromString(input)); - - input = null; - 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; @@ -195,83 +126,25 @@ public void fromDayTimeStringTest() { } } - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - } - @Test public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); + CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); } @Test public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } + CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - String input1 = "interval " + number + " " + unit; - String input2 = "interval " + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); + input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); } } 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 fdb81a06d41c9..72aa682bb95bc 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 @@ -19,7 +19,7 @@ package org.apache.spark.unsafe.types import org.apache.commons.text.similarity.LevenshteinDistance import org.scalacheck.{Arbitrary, Gen} -import org.scalatest.prop.GeneratorDrivenPropertyChecks +import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks // scalastyle:off import org.scalatest.{FunSuite, Matchers} @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.UTF8String.{fromString => toUTF8} /** * This TestSuite utilize ScalaCheck to generate randomized inputs for UTF8String testing. */ -class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenPropertyChecks with Matchers { +class UTF8StringPropertyCheckSuite extends FunSuite with ScalaCheckDrivenPropertyChecks with Matchers { // scalastyle:on test("toString") { diff --git a/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt b/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt index f43e083060e34..e944111ff9e93 100644 --- a/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt +++ b/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt @@ -6,35 +6,35 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Coalesced RDD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Coalesce Num Partitions: 100 Num Hosts: 1 455 623 151 0.2 4547.4 1.0X -Coalesce Num Partitions: 100 Num Hosts: 5 340 409 65 0.3 3397.1 1.3X -Coalesce Num Partitions: 100 Num Hosts: 10 292 363 95 0.3 2923.3 1.6X -Coalesce Num Partitions: 100 Num Hosts: 20 307 320 11 0.3 3069.8 1.5X -Coalesce Num Partitions: 100 Num Hosts: 40 333 368 55 0.3 3329.1 1.4X -Coalesce Num Partitions: 100 Num Hosts: 80 286 338 63 0.3 2862.5 1.6X -Coalesce Num Partitions: 500 Num Hosts: 1 769 837 59 0.1 7693.5 0.6X -Coalesce Num Partitions: 500 Num Hosts: 5 427 461 31 0.2 4268.5 1.1X -Coalesce Num Partitions: 500 Num Hosts: 10 372 389 27 0.3 3722.2 1.2X -Coalesce Num Partitions: 500 Num Hosts: 20 347 365 31 0.3 3468.5 1.3X -Coalesce Num Partitions: 500 Num Hosts: 40 335 336 1 0.3 3347.3 1.4X -Coalesce Num Partitions: 500 Num Hosts: 80 329 360 49 0.3 3294.5 1.4X -Coalesce Num Partitions: 1000 Num Hosts: 1 1254 1292 47 0.1 12538.6 0.4X -Coalesce Num Partitions: 1000 Num Hosts: 5 518 553 47 0.2 5177.0 0.9X -Coalesce Num Partitions: 1000 Num Hosts: 10 394 432 42 0.3 3937.3 1.2X -Coalesce Num Partitions: 1000 Num Hosts: 20 341 381 44 0.3 3414.4 1.3X -Coalesce Num Partitions: 1000 Num Hosts: 40 313 358 48 0.3 3134.9 1.5X -Coalesce Num Partitions: 1000 Num Hosts: 80 335 360 38 0.3 3347.0 1.4X -Coalesce Num Partitions: 5000 Num Hosts: 1 3937 4066 156 0.0 39375.0 0.1X -Coalesce Num Partitions: 5000 Num Hosts: 5 1413 1453 40 0.1 14133.4 0.3X -Coalesce Num Partitions: 5000 Num Hosts: 10 826 861 49 0.1 8255.2 0.6X -Coalesce Num Partitions: 5000 Num Hosts: 20 542 609 58 0.2 5423.3 0.8X -Coalesce Num Partitions: 5000 Num Hosts: 40 410 470 64 0.2 4101.0 1.1X -Coalesce Num Partitions: 5000 Num Hosts: 80 352 427 69 0.3 3515.3 1.3X -Coalesce Num Partitions: 10000 Num Hosts: 1 7101 7151 54 0.0 71007.4 0.1X -Coalesce Num Partitions: 10000 Num Hosts: 5 2540 2582 59 0.0 25396.2 0.2X -Coalesce Num Partitions: 10000 Num Hosts: 10 1378 1432 48 0.1 13781.4 0.3X -Coalesce Num Partitions: 10000 Num Hosts: 20 829 867 66 0.1 8286.8 0.5X -Coalesce Num Partitions: 10000 Num Hosts: 40 573 630 49 0.2 5730.2 0.8X -Coalesce Num Partitions: 10000 Num Hosts: 80 438 449 9 0.2 4382.5 1.0X +Coalesce Num Partitions: 100 Num Hosts: 1 344 360 14 0.3 3441.4 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 283 301 22 0.4 2825.1 1.2X +Coalesce Num Partitions: 100 Num Hosts: 10 270 271 2 0.4 2700.5 1.3X +Coalesce Num Partitions: 100 Num Hosts: 20 272 273 1 0.4 2721.1 1.3X +Coalesce Num Partitions: 100 Num Hosts: 40 271 272 1 0.4 2710.0 1.3X +Coalesce Num Partitions: 100 Num Hosts: 80 266 267 2 0.4 2656.3 1.3X +Coalesce Num Partitions: 500 Num Hosts: 1 609 619 15 0.2 6089.0 0.6X +Coalesce Num Partitions: 500 Num Hosts: 5 338 343 6 0.3 3383.0 1.0X +Coalesce Num Partitions: 500 Num Hosts: 10 303 306 3 0.3 3029.4 1.1X +Coalesce Num Partitions: 500 Num Hosts: 20 286 288 2 0.4 2855.9 1.2X +Coalesce Num Partitions: 500 Num Hosts: 40 279 282 4 0.4 2793.3 1.2X +Coalesce Num Partitions: 500 Num Hosts: 80 273 275 3 0.4 2725.9 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 1 951 955 4 0.1 9514.1 0.4X +Coalesce Num Partitions: 1000 Num Hosts: 5 421 429 8 0.2 4211.3 0.8X +Coalesce Num Partitions: 1000 Num Hosts: 10 347 352 4 0.3 3473.5 1.0X +Coalesce Num Partitions: 1000 Num Hosts: 20 309 312 5 0.3 3087.5 1.1X +Coalesce Num Partitions: 1000 Num Hosts: 40 290 294 6 0.3 2896.4 1.2X +Coalesce Num Partitions: 1000 Num Hosts: 80 281 286 5 0.4 2811.3 1.2X +Coalesce Num Partitions: 5000 Num Hosts: 1 3928 3950 27 0.0 39278.0 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 5 1373 1389 27 0.1 13725.2 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 10 812 827 13 0.1 8123.3 0.4X +Coalesce Num Partitions: 5000 Num Hosts: 20 530 540 9 0.2 5299.1 0.6X +Coalesce Num Partitions: 5000 Num Hosts: 40 421 425 5 0.2 4210.5 0.8X +Coalesce Num Partitions: 5000 Num Hosts: 80 335 344 12 0.3 3353.7 1.0X +Coalesce Num Partitions: 10000 Num Hosts: 1 7116 7120 4 0.0 71159.0 0.0X +Coalesce Num Partitions: 10000 Num Hosts: 5 2539 2598 51 0.0 25390.1 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 10 1393 1432 34 0.1 13928.1 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 20 833 1009 303 0.1 8329.2 0.4X +Coalesce Num Partitions: 10000 Num Hosts: 40 562 563 3 0.2 5615.2 0.6X +Coalesce Num Partitions: 10000 Num Hosts: 80 420 426 7 0.2 4204.0 0.8X diff --git a/core/benchmarks/CoalescedRDDBenchmark-results.txt b/core/benchmarks/CoalescedRDDBenchmark-results.txt index dd63b0adea4f2..f1b867951a074 100644 --- a/core/benchmarks/CoalescedRDDBenchmark-results.txt +++ b/core/benchmarks/CoalescedRDDBenchmark-results.txt @@ -2,39 +2,39 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +Coalesce Num Partitions: 100 Num Hosts: 1 395 401 9 0.3 3952.3 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 296 344 42 0.3 2963.2 1.3X +Coalesce Num Partitions: 100 Num Hosts: 10 294 308 15 0.3 2941.7 1.3X +Coalesce Num Partitions: 100 Num Hosts: 20 316 328 13 0.3 3155.2 1.3X +Coalesce Num Partitions: 100 Num Hosts: 40 294 316 36 0.3 2940.3 1.3X +Coalesce Num Partitions: 100 Num Hosts: 80 292 324 30 0.3 2922.2 1.4X +Coalesce Num Partitions: 500 Num Hosts: 1 629 687 61 0.2 6292.4 0.6X +Coalesce Num Partitions: 500 Num Hosts: 5 354 378 42 0.3 3541.7 1.1X +Coalesce Num Partitions: 500 Num Hosts: 10 318 338 29 0.3 3179.8 1.2X +Coalesce Num Partitions: 500 Num Hosts: 20 306 317 11 0.3 3059.2 1.3X +Coalesce Num Partitions: 500 Num Hosts: 40 294 311 28 0.3 2941.6 1.3X +Coalesce Num Partitions: 500 Num Hosts: 80 288 309 34 0.3 2883.9 1.4X +Coalesce Num Partitions: 1000 Num Hosts: 1 956 978 20 0.1 9562.2 0.4X +Coalesce Num Partitions: 1000 Num Hosts: 5 431 452 36 0.2 4306.2 0.9X +Coalesce Num Partitions: 1000 Num Hosts: 10 358 379 23 0.3 3581.1 1.1X +Coalesce Num Partitions: 1000 Num Hosts: 20 324 347 20 0.3 3236.7 1.2X +Coalesce Num Partitions: 1000 Num Hosts: 40 312 333 20 0.3 3116.8 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 80 307 342 32 0.3 3068.4 1.3X +Coalesce Num Partitions: 5000 Num Hosts: 1 3895 3906 12 0.0 38946.8 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 5 1388 1401 19 0.1 13881.7 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 10 806 839 57 0.1 8063.7 0.5X +Coalesce Num Partitions: 5000 Num Hosts: 20 546 573 44 0.2 5462.6 0.7X +Coalesce Num Partitions: 5000 Num Hosts: 40 413 418 5 0.2 4134.7 1.0X +Coalesce Num Partitions: 5000 Num Hosts: 80 345 365 23 0.3 3448.1 1.1X +Coalesce Num Partitions: 10000 Num Hosts: 1 6933 6966 55 0.0 69328.8 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 5 2455 2499 69 0.0 24551.7 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 10 1352 1392 34 0.1 13520.2 0.3X +Coalesce Num Partitions: 10000 Num Hosts: 20 815 853 50 0.1 8147.5 0.5X +Coalesce Num Partitions: 10000 Num Hosts: 40 558 581 28 0.2 5578.0 0.7X +Coalesce Num Partitions: 10000 Num Hosts: 80 416 423 5 0.2 4163.3 0.9X diff --git a/core/benchmarks/KryoBenchmark-jdk11-results.txt b/core/benchmarks/KryoBenchmark-jdk11-results.txt index 0afa06519d2f4..27f0b8f59f47a 100644 --- a/core/benchmarks/KryoBenchmark-jdk11-results.txt +++ b/core/benchmarks/KryoBenchmark-jdk11-results.txt @@ -6,23 +6,23 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Benchmark Kryo Unsafe vs safe Serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -basicTypes: Int with unsafe:true 324 329 2 3.1 324.0 1.0X -basicTypes: Long with unsafe:true 353 355 1 2.8 353.0 0.9X -basicTypes: Float with unsafe:true 336 338 1 3.0 336.4 1.0X -basicTypes: Double with unsafe:true 347 347 1 2.9 346.5 0.9X -Array: Int with unsafe:true 4 4 0 265.4 3.8 86.0X -Array: Long with unsafe:true 6 7 0 157.3 6.4 51.0X -Array: Float with unsafe:true 4 4 0 268.8 3.7 87.1X -Array: Double with unsafe:true 6 7 0 157.5 6.3 51.0X -Map of string->Double with unsafe:true 52 52 1 19.3 51.8 6.3X -basicTypes: Int with unsafe:false 357 358 1 2.8 357.2 0.9X -basicTypes: Long with unsafe:false 387 388 0 2.6 387.4 0.8X -basicTypes: Float with unsafe:false 356 357 1 2.8 356.0 0.9X -basicTypes: Double with unsafe:false 371 372 1 2.7 371.0 0.9X -Array: Int with unsafe:false 24 24 0 41.3 24.2 13.4X -Array: Long with unsafe:false 37 38 0 26.8 37.4 8.7X -Array: Float with unsafe:false 11 11 0 94.9 10.5 30.8X -Array: Double with unsafe:false 18 18 0 55.2 18.1 17.9X -Map of string->Double with unsafe:false 55 55 0 18.2 55.1 5.9X +basicTypes: Int with unsafe:true 275 288 14 3.6 275.2 1.0X +basicTypes: Long with unsafe:true 331 336 13 3.0 330.9 0.8X +basicTypes: Float with unsafe:true 304 305 1 3.3 304.4 0.9X +basicTypes: Double with unsafe:true 328 332 3 3.0 328.1 0.8X +Array: Int with unsafe:true 4 4 0 252.8 4.0 69.6X +Array: Long with unsafe:true 6 6 0 161.5 6.2 44.5X +Array: Float with unsafe:true 4 4 0 264.6 3.8 72.8X +Array: Double with unsafe:true 6 7 0 160.5 6.2 44.2X +Map of string->Double with unsafe:true 52 52 0 19.3 51.8 5.3X +basicTypes: Int with unsafe:false 344 345 1 2.9 344.3 0.8X +basicTypes: Long with unsafe:false 372 373 1 2.7 372.3 0.7X +basicTypes: Float with unsafe:false 333 334 1 3.0 333.4 0.8X +basicTypes: Double with unsafe:false 344 345 0 2.9 344.3 0.8X +Array: Int with unsafe:false 25 25 0 40.8 24.5 11.2X +Array: Long with unsafe:false 37 37 1 27.3 36.7 7.5X +Array: Float with unsafe:false 11 11 0 92.1 10.9 25.4X +Array: Double with unsafe:false 17 18 0 58.3 17.2 16.0X +Map of string->Double with unsafe:false 51 52 1 19.4 51.5 5.3X diff --git a/core/benchmarks/KryoBenchmark-results.txt b/core/benchmarks/KryoBenchmark-results.txt index 91e22f3afc14f..49791e6e87e3a 100644 --- a/core/benchmarks/KryoBenchmark-results.txt +++ b/core/benchmarks/KryoBenchmark-results.txt @@ -2,28 +2,27 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Benchmark Kryo Unsafe vs safe Serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +basicTypes: Int with unsafe:true 269 290 23 3.7 269.0 1.0X +basicTypes: Long with unsafe:true 294 295 1 3.4 293.8 0.9X +basicTypes: Float with unsafe:true 300 301 1 3.3 300.4 0.9X +basicTypes: Double with unsafe:true 304 305 1 3.3 304.0 0.9X +Array: Int with unsafe:true 5 6 1 193.5 5.2 52.0X +Array: Long with unsafe:true 8 9 1 131.2 7.6 35.3X +Array: Float with unsafe:true 6 6 0 163.5 6.1 44.0X +Array: Double with unsafe:true 9 10 0 108.8 9.2 29.3X +Map of string->Double with unsafe:true 54 54 1 18.7 53.6 5.0X +basicTypes: Int with unsafe:false 326 327 1 3.1 326.2 0.8X +basicTypes: Long with unsafe:false 353 354 1 2.8 353.3 0.8X +basicTypes: Float with unsafe:false 325 327 1 3.1 325.1 0.8X +basicTypes: Double with unsafe:false 335 336 1 3.0 335.0 0.8X +Array: Int with unsafe:false 27 28 1 36.7 27.2 9.9X +Array: Long with unsafe:false 40 41 1 25.0 40.0 6.7X +Array: Float with unsafe:false 12 13 1 80.8 12.4 21.7X +Array: Double with unsafe:false 21 21 1 48.6 20.6 13.1X +Map of string->Double with unsafe:false 56 57 1 17.8 56.1 4.8X diff --git a/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt b/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt index 0e6d32649840f..6b148bde12d36 100644 --- a/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt +++ b/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt @@ -6,7 +6,7 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Benchmark KryoPool vs old"pool of 1" implementation: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -KryoPool:true 6524 9149 NaN 0.0 13047076.5 1.0X -KryoPool:false 12855 16469 663 0.0 25709170.6 0.5X +KryoPool:true 6208 8374 NaN 0.0 12416876.6 1.0X +KryoPool:false 9084 11577 724 0.0 18168947.4 0.7X diff --git a/core/benchmarks/KryoSerializerBenchmark-results.txt b/core/benchmarks/KryoSerializerBenchmark-results.txt index c3ce336d93241..609f3298cbc00 100644 --- a/core/benchmarks/KryoSerializerBenchmark-results.txt +++ b/core/benchmarks/KryoSerializerBenchmark-results.txt @@ -1,12 +1,12 @@ ================================================================================================ -Benchmark KryoPool vs "pool of 1" +Benchmark KryoPool vs old"pool of 1" implementation ================================================================================================ -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Benchmark KryoPool vs old"pool of 1" implementation: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +KryoPool:true 6012 7586 NaN 0.0 12023020.2 1.0X +KryoPool:false 9289 11566 909 0.0 18578683.1 0.6X diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..db23cf5c12ea7 --- /dev/null +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt @@ -0,0 +1,66 @@ +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 170 178 9 1.2 849.7 1.0X +Deserialization 530 535 9 0.4 2651.1 0.3X + +Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized Broadcast MapStatus sizes: 2 MB + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 157 165 7 1.3 785.4 1.0X +Deserialization 495 588 79 0.4 2476.7 0.3X + +Compressed Serialized MapStatus sizes: 2 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 344 351 4 0.6 1720.4 1.0X +Deserialization 527 579 99 0.4 2635.9 0.7X + +Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized Broadcast MapStatus sizes: 13 MB + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 317 321 4 0.6 1583.8 1.0X +Deserialization 530 540 15 0.4 2648.3 0.6X + +Compressed Serialized MapStatus sizes: 13 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1738 1849 156 0.1 8692.0 1.0X +Deserialization 946 977 33 0.2 4730.2 1.8X + +Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized Broadcast MapStatus sizes: 121 MB + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1379 1432 76 0.1 6892.6 1.0X +Deserialization 929 941 19 0.2 4645.5 1.5X + +Compressed Serialized MapStatus sizes: 121 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt new file mode 100644 index 0000000000000..053f4bf771923 --- /dev/null +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -0,0 +1,66 @@ +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 178 187 15 1.1 887.5 1.0X +Deserialization 530 558 32 0.4 2647.5 0.3X + +Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized Broadcast MapStatus sizes: 2 MB + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 167 175 7 1.2 835.7 1.0X +Deserialization 523 537 22 0.4 2616.2 0.3X + +Compressed Serialized MapStatus sizes: 2 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 351 416 147 0.6 1754.4 1.0X +Deserialization 546 551 8 0.4 2727.6 0.6X + +Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized Broadcast MapStatus sizes: 13 MB + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 320 321 1 0.6 1598.0 1.0X +Deserialization 542 549 7 0.4 2709.0 0.6X + +Compressed Serialized MapStatus sizes: 13 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1671 1877 290 0.1 8357.3 1.0X +Deserialization 943 970 32 0.2 4715.8 1.8X + +Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized Broadcast MapStatus sizes: 121 MB + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1373 1436 89 0.1 6865.0 1.0X +Deserialization 940 970 37 0.2 4699.1 1.5X + +Compressed Serialized MapStatus sizes: 121 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + diff --git a/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt b/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt index 1cd08a7748109..605b856d53382 100644 --- a/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt +++ b/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt @@ -6,35 +6,35 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Empty Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.1 13755.0 1.0X -Utils.cloneProperties 0 0 0 3.5 285.0 48.3X +SerializationUtils.clone 0 0 0 0.1 11539.0 1.0X +Utils.cloneProperties 0 0 0 1.7 572.0 20.2X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz System Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 191892.0 1.0X -Utils.cloneProperties 0 0 0 0.2 6027.0 31.8X +SerializationUtils.clone 0 0 0 0.0 217514.0 1.0X +Utils.cloneProperties 0 0 0 0.2 5387.0 40.4X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Small Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 1 1 0 0.0 721334.0 1.0X -Utils.cloneProperties 0 0 0 0.2 5237.0 137.7X +SerializationUtils.clone 1 1 0 0.0 634574.0 1.0X +Utils.cloneProperties 0 0 0 0.3 3082.0 205.9X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Medium Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 3 3 0 0.0 3006554.0 1.0X -Utils.cloneProperties 0 0 0 0.0 27042.0 111.2X +SerializationUtils.clone 3 3 0 0.0 2576565.0 1.0X +Utils.cloneProperties 0 0 0 0.1 16071.0 160.3X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Large Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 6 6 0 0.0 5864452.0 1.0X -Utils.cloneProperties 0 0 0 0.0 53760.0 109.1X +SerializationUtils.clone 5 5 0 0.0 5027248.0 1.0X +Utils.cloneProperties 0 0 0 0.0 31842.0 157.9X diff --git a/core/benchmarks/PropertiesCloneBenchmark-results.txt b/core/benchmarks/PropertiesCloneBenchmark-results.txt index 00c9561648be0..5d332a147c698 100644 --- a/core/benchmarks/PropertiesCloneBenchmark-results.txt +++ b/core/benchmarks/PropertiesCloneBenchmark-results.txt @@ -2,39 +2,39 @@ Properties Cloning ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 -Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Empty Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.2 4184.0 1.0X -Utils.cloneProperties 0 0 0 55.6 18.0 232.4X +SerializationUtils.clone 0 0 0 0.1 13640.0 1.0X +Utils.cloneProperties 0 0 0 1.6 608.0 22.4X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 -Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz System Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 107612.0 1.0X -Utils.cloneProperties 0 0 0 1.0 962.0 111.9X +SerializationUtils.clone 0 0 0 0.0 238968.0 1.0X +Utils.cloneProperties 0 0 0 0.4 2318.0 103.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 -Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Small Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 330210.0 1.0X -Utils.cloneProperties 0 0 0 0.9 1082.0 305.2X +SerializationUtils.clone 1 1 0 0.0 725849.0 1.0X +Utils.cloneProperties 0 0 0 0.3 2900.0 250.3X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 -Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Medium Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 1 2 0 0.0 1336301.0 1.0X -Utils.cloneProperties 0 0 0 0.2 5456.0 244.9X +SerializationUtils.clone 3 3 0 0.0 2999676.0 1.0X +Utils.cloneProperties 0 0 0 0.1 11734.0 255.6X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 -Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Large Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 3 3 0 0.0 2634336.0 1.0X -Utils.cloneProperties 0 0 0 0.1 10822.0 243.4X +SerializationUtils.clone 6 6 1 0.0 5846410.0 1.0X +Utils.cloneProperties 0 0 0 0.0 22405.0 260.9X diff --git a/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt b/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt index 3b6091cc740ac..9aa10e4835a2f 100644 --- a/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt +++ b/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt @@ -6,29 +6,29 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz nextInt: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 1357 1358 0 73.7 13.6 1.0X -XORShiftRandom 228 228 0 438.0 2.3 5.9X +java.util.Random 1362 1362 0 73.4 13.6 1.0X +XORShiftRandom 227 227 0 440.6 2.3 6.0X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz nextLong: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 2718 2719 1 36.8 27.2 1.0X -XORShiftRandom 632 633 0 158.1 6.3 4.3X +java.util.Random 2725 2726 1 36.7 27.3 1.0X +XORShiftRandom 694 694 1 144.1 6.9 3.9X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz nextDouble: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 2722 2723 0 36.7 27.2 1.0X -XORShiftRandom 632 632 0 158.3 6.3 4.3X +java.util.Random 2727 2728 0 36.7 27.3 1.0X +XORShiftRandom 693 694 0 144.2 6.9 3.9X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz nextGaussian: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 6979 6979 1 14.3 69.8 1.0X -XORShiftRandom 5183 5183 0 19.3 51.8 1.3X +java.util.Random 7012 7016 4 14.3 70.1 1.0X +XORShiftRandom 6065 6067 1 16.5 60.7 1.2X ================================================================================================ @@ -39,6 +39,6 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Hash seed: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -XORShiftRandom.hashSeed 38 39 1 263.9 3.8 1.0X +XORShiftRandom.hashSeed 36 37 1 276.5 3.6 1.0X diff --git a/core/benchmarks/XORShiftRandomBenchmark-results.txt b/core/benchmarks/XORShiftRandomBenchmark-results.txt index 1140489e4a7f3..4b069878b2e9b 100644 --- a/core/benchmarks/XORShiftRandomBenchmark-results.txt +++ b/core/benchmarks/XORShiftRandomBenchmark-results.txt @@ -2,43 +2,43 @@ Pseudo random ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +nextInt: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 1362 1396 59 73.4 13.6 1.0X +XORShiftRandom 227 227 0 440.7 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +nextLong: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 2732 2732 1 36.6 27.3 1.0X +XORShiftRandom 630 630 1 158.7 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +nextDouble: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 2731 2732 1 36.6 27.3 1.0X +XORShiftRandom 630 630 0 158.8 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +nextGaussian: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 8895 8899 4 11.2 88.9 1.0X +XORShiftRandom 5049 5052 5 19.8 50.5 1.8X ================================================================================================ hash seed ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Hash seed: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +XORShiftRandom.hashSeed 67 68 1 148.8 6.7 1.0X diff --git a/core/pom.xml b/core/pom.xml index 42fc2c4b3a287..38eb8adac500e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -384,6 +384,11 @@ curator-test test + + org.apache.hadoop + hadoop-minikdc + test + net.razorvine pyrolite @@ -551,6 +556,15 @@ + + scala-2.13 + + + org.scala-lang.modules + scala-parallel-collections_${scala.binary.version} + + + 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 7b68b399e6e84..9a9d0c7946549 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -54,7 +54,7 @@ public MemoryMode getMode() { /** * Returns the size of used memory in bytes. */ - protected long getUsed() { + public long getUsed() { return used; } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index e9e50ecc11e52..e4554bda8acab 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -46,4 +46,10 @@ public interface ShuffleDataIO { * are only invoked on the executors. */ ShuffleExecutorComponents executor(); + + /** + * Called once on driver process to bootstrap the shuffle metadata modules that + * are maintained by the driver. + */ + ShuffleDriverComponents driver(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java new file mode 100644 index 0000000000000..b4cec17b85b32 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java @@ -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.shuffle.api; + +import java.util.Map; + +import org.apache.spark.annotation.Private; + +/** + * :: Private :: + * An interface for building shuffle support modules for the Driver. + */ +@Private +public interface ShuffleDriverComponents { + + /** + * Called once in the driver to bootstrap this module that is specific to this application. + * This method is called before submitting executor requests to the cluster manager. + * + * This method should prepare the module with its shuffle components i.e. registering against + * an external file servers or shuffle services, or creating tables in a shuffle + * storage data database. + * + * @return additional SparkConf settings necessary for initializing the executor components. + * This would include configurations that cannot be statically set on the application, like + * the host:port of external services for shuffle storage. + */ + Map initializeApplication(); + + /** + * Called once at the end of the Spark application to clean up any existing shuffle state. + */ + void cleanupApplication(); + + /** + * Called once per shuffle id when the shuffle id is first generated for a shuffle stage. + * + * @param shuffleId The unique identifier for the shuffle stage. + */ + default void registerShuffle(int shuffleId) {} + + /** + * Removes shuffle data associated with the given shuffle. + * + * @param shuffleId The unique identifier for the shuffle stage. + * @param blocking Whether this call should block on the deletion of the data. + */ + default void removeShuffle(int shuffleId, boolean blocking) {} +} diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 804119cd06fa6..30ca177545789 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.api; import java.io.IOException; +import java.util.Map; import java.util.Optional; import org.apache.spark.annotation.Private; @@ -34,25 +35,26 @@ public interface ShuffleExecutorComponents { /** * Called once per executor to bootstrap this module with state that is specific to * that executor, specifically the application ID and executor ID. + * + * @param appId The Spark application id + * @param execId The unique identifier of the executor being initialized + * @param extraConfigs Extra configs that were returned by + * {@link ShuffleDriverComponents#initializeApplication()} */ - void initializeExecutor(String appId, String execId); + void initializeExecutor(String appId, String execId, Map extraConfigs); /** * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Within the shuffle, the identifier of the map task - * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * @param mapTaskId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId, + long mapTaskId, int numPartitions) throws IOException; /** @@ -64,15 +66,11 @@ ShuffleMapOutputWriter createMapOutputWriter( * preserving an optimization in the local disk shuffle storage implementation. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Within the shuffle, the identifier of the map task - * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * @param mapId An ID of the map task. The ID is unique within this Spark application. */ default Optional createSingleFileMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId) throws IOException { + long mapId) throws IOException { return Optional.empty(); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 7fac00b7fbc3f..21abe9a57cd25 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -39,7 +39,7 @@ public interface ShuffleMapOutputWriter { * for the same partition within any given map task. The partition identifier will be in the * range of precisely 0 (inclusive) to numPartitions (exclusive), where numPartitions was * provided upon the creation of this map output writer via - * {@link ShuffleExecutorComponents#createMapOutputWriter(int, int, long, int)}. + * {@link ShuffleExecutorComponents#createMapOutputWriter(int, long, int)}. *

* Calls to this method will be invoked with monotonically increasing reducePartitionIds; each * call to this method will be called with a reducePartitionId that is strictly greater than 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 f75e932860f90..dc157eaa3b253 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 @@ -85,8 +85,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; - private final int mapId; - private final long mapTaskAttemptId; + private final long mapId; private final Serializer serializer; private final ShuffleExecutorComponents shuffleExecutorComponents; @@ -106,8 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, BypassMergeSortShuffleHandle handle, - int mapId, - long mapTaskAttemptId, + long mapId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, ShuffleExecutorComponents shuffleExecutorComponents) { @@ -117,7 +115,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); this.mapId = mapId; - this.mapTaskAttemptId = mapTaskAttemptId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); @@ -130,11 +127,12 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions); + .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { partitionLengths = mapOutputWriter.commitAllPartitions(); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -167,7 +165,8 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapId); } catch (Exception e) { try { mapOutputWriter.abort(e); 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 4d11abd36985e..d09282e61a9c7 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 @@ -78,7 +78,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; - private final int mapId; + private final long mapId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -109,7 +109,7 @@ public UnsafeShuffleWriter( BlockManager blockManager, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, - int mapId, + long mapId, TaskContext taskContext, SparkConf sparkConf, ShuffleWriteMetricsReporter writeMetrics, @@ -228,7 +228,8 @@ void closeAndWriteOutput() throws IOException { } } } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapId); } @VisibleForTesting @@ -264,16 +265,11 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { long[] partitionLengths; if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter( - shuffleId, - mapId, - taskContext.taskAttemptId(), - partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); return mapWriter.commitAllPartitions(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = - shuffleExecutorComponents.createSingleFileMapOutputWriter( - shuffleId, mapId, taskContext.taskAttemptId()); + shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); if (maybeSingleFileWriter.isPresent()) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. @@ -298,11 +294,7 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter( - shuffleId, - mapId, - taskContext.taskAttemptId(), - partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); try { // There are multiple spills to merge, so none of these spill files' lengths were counted // towards our shuffle write count or shuffle write time. If we use the slow merge path, diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java index cabcb171ac23a..50eb2f1813714 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java @@ -18,8 +18,9 @@ package org.apache.spark.shuffle.sort.io; import org.apache.spark.SparkConf; -import org.apache.spark.shuffle.api.ShuffleExecutorComponents; import org.apache.spark.shuffle.api.ShuffleDataIO; +import org.apache.spark.shuffle.api.ShuffleDriverComponents; +import org.apache.spark.shuffle.api.ShuffleExecutorComponents; /** * Implementation of the {@link ShuffleDataIO} plugin system that replicates the local shuffle @@ -37,4 +38,9 @@ public LocalDiskShuffleDataIO(SparkConf sparkConf) { public ShuffleExecutorComponents executor() { return new LocalDiskShuffleExecutorComponents(sparkConf); } + + @Override + public ShuffleDriverComponents driver() { + return new LocalDiskShuffleDriverComponents(); + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java new file mode 100644 index 0000000000000..92b4b318c552d --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java @@ -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.shuffle.sort.io; + +import java.util.Collections; +import java.util.Map; + +import org.apache.spark.SparkEnv; +import org.apache.spark.shuffle.api.ShuffleDriverComponents; +import org.apache.spark.storage.BlockManagerMaster; + +public class LocalDiskShuffleDriverComponents implements ShuffleDriverComponents { + + private BlockManagerMaster blockManagerMaster; + + @Override + public Map initializeApplication() { + blockManagerMaster = SparkEnv.get().blockManager().master(); + return Collections.emptyMap(); + } + + @Override + public void cleanupApplication() { + // nothing to clean up + } + + @Override + public void removeShuffle(int shuffleId, boolean blocking) { + if (blockManagerMaster == null) { + throw new IllegalStateException("Driver components must be initialized before using"); + } + blockManagerMaster.removeShuffle(shuffleId, blocking); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 47aa2e39fe29b..eb4d9d9abc8e3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.sort.io; +import java.util.Map; import java.util.Optional; import com.google.common.annotations.VisibleForTesting; @@ -50,7 +51,7 @@ public LocalDiskShuffleExecutorComponents( } @Override - public void initializeExecutor(String appId, String execId) { + public void initializeExecutor(String appId, String execId, Map extraConfigs) { blockManager = SparkEnv.get().blockManager(); if (blockManager == null) { throw new IllegalStateException("No blockManager available from the SparkEnv."); @@ -61,22 +62,20 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId, + long mapTaskId, int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, blockResolver, sparkConf); + shuffleId, mapTaskId, numPartitions, blockResolver, sparkConf); } @Override public Optional createSingleFileMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId) { + long mapId) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 444cdc4270ecd..a6529fd76188a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -48,7 +48,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; - private final int mapId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -64,7 +64,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { public LocalDiskShuffleMapOutputWriter( int shuffleId, - int mapId, + long mapId, int numPartitions, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java index 6b0a797a61b52..c8b41992a8919 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java @@ -29,12 +29,12 @@ public class LocalDiskSingleSpillMapOutputWriter implements SingleSpillShuffleMapOutputWriter { private final int shuffleId; - private final int mapId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; public LocalDiskSingleSpillMapOutputWriter( int shuffleId, - int mapId, + long mapId, IndexShuffleBlockResolver blockResolver) { this.shuffleId = shuffleId; this.mapId = mapId; 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 d320ba3139541..b15365fe54ad6 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 @@ -886,6 +886,7 @@ public void reset() { numKeys = 0; numValues = 0; freeArray(longArray); + longArray = null; while (dataPages.size() > 0) { MemoryBlock dataPage = dataPages.removeLast(); freePage(dataPage); 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 eb12848900b58..71652d0e9f5e8 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -33,7 +33,8 @@ 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 -# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support +# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs +# in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 31ef04552b880..4c06ddf3b31f9 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 @@ -37,8 +37,8 @@

Summary

Disk Used Cores - Active Tasks - Failed Tasks + Active Tasks + Failed Tasks Complete Tasks Total Tasks Executors Disk Used Cores Resources - Active Tasks - Failed Tasks + Active Tasks + Failed Tasks Complete Tasks Total Tasks diff --git a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js index 9960d5c34d1fc..ecd580e5c64aa 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js +++ b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js @@ -97,9 +97,14 @@ sorttable = { sorttable.reverse(this.sorttable_tbody); this.className = this.className.replace('sorttable_sorted', 'sorttable_sorted_reverse'); - this.removeChild(document.getElementById('sorttable_sortfwdind')); + rowlists = this.parentNode.getElementsByTagName("span"); + for (var j=0; j < rowlists.length; j++) { + if (rowlists[j].className.search(/\bsorttable_sortfwdind\b/)) { + rowlists[j].parentNode.removeChild(rowlists[j]); + } + } sortrevind = document.createElement('span'); - sortrevind.id = "sorttable_sortrevind"; + sortrevind.class = "sorttable_sortrevind"; sortrevind.innerHTML = stIsIE ? ' 5' : ' ▾'; this.appendChild(sortrevind); return; @@ -110,9 +115,14 @@ sorttable = { sorttable.reverse(this.sorttable_tbody); this.className = this.className.replace('sorttable_sorted_reverse', 'sorttable_sorted'); - this.removeChild(document.getElementById('sorttable_sortrevind')); + rowlists = this.parentNode.getElementsByTagName("span"); + for (var j=0; j < rowlists.length; j++) { + if (rowlists[j].className.search(/\sorttable_sortrevind\b/)) { + rowlists[j].parentNode.removeChild(rowlists[j]); + } + } sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.class = "sorttable_sortfwdind"; sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▴'; this.appendChild(sortfwdind); return; @@ -126,14 +136,17 @@ sorttable = { cell.className = cell.className.replace('sorttable_sorted',''); } }); - sortfwdind = document.getElementById('sorttable_sortfwdind'); - if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } - sortrevind = document.getElementById('sorttable_sortrevind'); - if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } + rowlists = this.parentNode.getElementsByTagName("span"); + for (var j=0; j < rowlists.length; j++) { + if (rowlists[j].className.search(/\bsorttable_sortfwdind\b/) + || rowlists[j].className.search(/\sorttable_sortrevind\b/) ) { + rowlists[j].parentNode.removeChild(rowlists[j]); + } + } this.className += ' sorttable_sorted'; sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.class = "sorttable_sortfwdind"; sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▴'; this.appendChild(sortfwdind); diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 3ef1a76fd7202..b28c981da20a5 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -286,7 +286,7 @@ $(document).ready(function () { " Show Additional Metrics" + "" + " +## Scalar Functions +(to be filled soon) + ## Aggregations The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index c55e589aa8c3f..b4f8d8be11c4f 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -179,6 +179,7 @@ Below is a list of all the keywords in Spark SQL. MONTHreservednon-reservedreserved MONTHSnon-reservednon-reservednon-reserved MSCKnon-reservednon-reservednon-reserved + NAMESPACEnon-reservednon-reservednon-reserved NAMESPACESnon-reservednon-reservednon-reserved NATURALreservedstrict-non-reservedreserved NOnon-reservednon-reservedreserved @@ -209,6 +210,7 @@ Below is a list of all the keywords in Spark SQL. PRECEDINGnon-reservednon-reservednon-reserved PRIMARYreservednon-reservedreserved PRINCIPALSnon-reservednon-reservednon-reserved + PROPERTIESnon-reservednon-reservednon-reserved PURGEnon-reservednon-reservednon-reserved QUERYnon-reservednon-reservednon-reserved RANGEnon-reservednon-reservedreserved @@ -280,6 +282,7 @@ Below is a list of all the keywords in Spark SQL. UNKNOWNreservednon-reservedreserved UNLOCKnon-reservednon-reservednon-reserved UNSETnon-reservednon-reservednon-reserved + UPDATEnon-reservednon-reservedreserved USEnon-reservednon-reservednon-reserved USERreservednon-reservedreserved USINGreservedstrict-non-reservedreserved diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 71e533eef5fe6..d03ca663e8e3f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -23,6 +23,7 @@ license: | {:toc} ## Upgrading from Spark SQL 2.4 to 3.0 + - Since Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception will be thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and earlier, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior. - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. @@ -201,6 +202,22 @@ license: | + - Since Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that will be converted to ordinary date or timestamp values when read. The following string values are supported for dates: + - `epoch [zoneId]` - 1970-01-01 + - `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone` + - `yesterday [zoneId]` - the current date - 1 + - `tomorrow [zoneId]` - the current date + 1 + - `now` - the date of running the current query. It has the same notion as today + For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values: + - `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero) + - `today [zoneId]` - midnight today + - `yesterday [zoneId]` - midnight yesterday + - `tomorrow [zoneId]` - midnight tomorrow + - `now` - current query start time + For example `SELECT timestamp 'tomorrow';`. + + - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. + ## 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 @@ -785,7 +802,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `CLUSTER BY` * `SORT BY` * All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Relational operators (`=`, `<=>`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) * Logical operators (`AND`, `&&`, `OR`, `||`, etc) * Complex type constructors diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index 25112d1d2a833..7f01483d40583 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -219,3 +219,20 @@ Note that a standard UDF (non-Pandas) will load timestamp data as Python datetim 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. + +### Compatibiliy Setting for PyArrow >= 0.15.0 and Spark 2.3.x, 2.4.x + +Since Arrow 0.15.0, a change in the binary IPC format requires an environment variable to be +compatible with previous versions of Arrow <= 0.14.1. This is only necessary to do for PySpark +users with versions 2.3.x and 2.4.x that have manually upgraded PyArrow to 0.15.0. The following +can be added to `conf/spark-env.sh` to use the legacy Arrow IPC format: + +``` +ARROW_PRE_0_15_IPC_FORMAT=1 +``` + +This will instruct PyArrow >= 0.15.0 to use the legacy IPC format with the older Arrow Java that +is in Spark 2.3.x and 2.4.x. Not setting this environment variable will lead to a similar error as +described in [SPARK-29367](https://issues.apache.org/jira/browse/SPARK-29367) when running +`pandas_udf`s or `toPandas()` with Arrow enabled. More information about the Arrow IPC change can +be read on the Arrow 0.15.0 release [blog](http://arrow.apache.org/blog/2019/10/06/0.15.0-release/#columnar-streaming-protocol-change-since-0140). diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md index f57b81ead6f49..f7ce84c774301 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md @@ -19,4 +19,27 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`ADD FILE` adds a file to the list of resources. The added file can be listed using [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html). + +### Syntax +{% highlight sql %} +ADD FILE file_name +{% endhighlight %} + +### Parameters +
+
file_name
+
The name of the file to be added.
+
+ +### Examples +{% highlight sql %} +ADD FILE /tmp/test; +{% endhighlight %} + +### Related Statements + * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) + * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) + * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) + diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index cd91119c8c2ba..411bb52604aa1 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -19,4 +19,26 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`ADD JAR` adds a JAR file to the list of resources. The added JAR file can be listed using [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html). + +### Syntax +{% highlight sql %} +ADD JAR file_name +{% endhighlight %} + +### Parameters +
+
file_name
+
The name of the JAR file to be added. It could be either on a local file system or a distributed file system.
+
+ +### Examples +{% highlight sql %} +ADD JAR /tmp/test.jar; +{% endhighlight %} + +### Related Statements + * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) + * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) + * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md new file mode 100644 index 0000000000000..4214d2fdb93eb --- /dev/null +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md @@ -0,0 +1,42 @@ +--- +layout: global +title: LIST FILE +displayTitle: LIST FILE +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description +`LIST FILE` lists the files added by [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html). + +### Syntax +{% highlight sql %} +LIST FILE +{% endhighlight %} + +### Examples +{% highlight sql %} +ADD FILE /tmp/test; +LIST FILE; +-- output for LIST FILE +file:/private/tmp/test +{% endhighlight %} + +### Related Statements + * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) + * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) + * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) + diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md new file mode 100644 index 0000000000000..29892747fc3a7 --- /dev/null +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md @@ -0,0 +1,42 @@ +--- +layout: global +title: LIST JAR +displayTitle: LIST JAR +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +### Description +`LIST JAR` lists the JARs added by [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html). + +### Syntax +{% highlight sql %} +LIST JAR +{% endhighlight %} + +### Examples +{% highlight sql %} +ADD JAR /tmp/test.jar; +LIST JAR; +-- output for LIST JAR +spark://192.168.1.112:62859/jars/test.jar +{% endhighlight %} + +### Related Statements + * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) + * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) + * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) + diff --git a/docs/sql-ref-syntax-aux-show-create-table.md b/docs/sql-ref-syntax-aux-show-create-table.md index 2cf40915774c4..7871d30b5b186 100644 --- a/docs/sql-ref-syntax-aux-show-create-table.md +++ b/docs/sql-ref-syntax-aux-show-create-table.md @@ -19,4 +19,46 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`SHOW CREATE TABLE` returns the [CREATE TABLE statement](sql-ref-syntax-ddl-create-table.html) or [CREATE VIEW statement](sql-ref-syntax-ddl-create-view.html) that was used to create a given table or view. `SHOW CREATE TABLE` on a non-existent table or a temporary view throws an exception. + +### Syntax +{% highlight sql %} +SHOW CREATE TABLE name +{% endhighlight %} + +### Parameters +
+
name
+
The name of the table or view to be used for SHOW CREATE TABLE.
+
+ +### Examples +{% highlight sql %} +CREATE TABLE test (c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + STORED AS TEXTFILE + TBLPROPERTIES ('prop1' = 'value1', 'prop2' = 'value2'); + +show create table test; + +-- the result of SHOW CREATE TABLE test +CREATE TABLE `test`(`c` INT) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +WITH SERDEPROPERTIES ( + 'field.delim' = ',', + 'serialization.format' = ',' +) +STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +TBLPROPERTIES ( + 'transient_lastDdlTime' = '1569350233', + 'prop1' = 'value1', + 'prop2' = 'value2' +) + +{% endhighlight %} + +### Related Statements + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md index e4340d608bf5e..46cfe40dc2600 100644 --- a/docs/sql-ref-syntax-aux-show-tables.md +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -18,5 +18,90 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +### Description -**This page is under construction** +The `SHOW TABLES` statement returns all the tables for an optionally specified database. +Additionally, the output of this statement may be filtered by an optional matching +pattern. If no database is specified then the tables are returned from the +current database. + +### Syntax +{% highlight sql %} +SHOW TABLES [{FROM|IN} database_name] [LIKE 'regex_pattern'] +{% endhighlight %} + +### Parameters +
+
{FROM|IN} database_name
+
+ Specifies the database name from which tables are listed. +
+
LIKE regex_pattern
+
+ Specifies the regular expression pattern that is used to filter out unwanted tables. +
    +
  • Except for `*` and `|` character, the pattern works like a regex.
  • +
  • `*` alone matches 0 or more characters and `|` is used to separate multiple different regexes, + any of which can match.
  • +
  • The leading and trailing blanks are trimmed in the input pattern before processing.
  • +
+ +
+
+ +### Example +{% highlight sql %} +-- List all tables in default database +SHOW TABLES; + +-----------+------------+--------------+--+ + | database | tableName | isTemporary | + +-----------+------------+--------------+--+ + | default | sam | false | + | default | sam1 | false | + | default | suj | false | + +-----------+------------+--------------+--+ + +-- List all tables from userdb database +SHOW TABLES FROM userdb; + +-----------+------------+--------------+--+ + | database | tableName | isTemporary | + +-----------+------------+--------------+--+ + | userdb | user1 | false | + | userdb | user2 | false | + +-----------+------------+--------------+--+ + +-- List all tables in userdb database +SHOW TABLES IN userdb; + +-----------+------------+--------------+--+ + | database | tableName | isTemporary | + +-----------+------------+--------------+--+ + | userdb | user1 | false | + | userdb | user2 | false | + +-----------+------------+--------------+--+ + +-- List all tables from default database matching the pattern `sam*` +SHOW TABLES FROM default LIKE 'sam*'; + +-----------+------------+--------------+--+ + | database | tableName | isTemporary | + +-----------+------------+--------------+--+ + | default | sam | false | + | default | sam1 | false | + +-----------+------------+--------------+--+ + +-- List all tables matching the pattern `sam*|suj` +SHOW TABLES LIKE 'sam*|suj'; + +-----------+------------+--------------+--+ + | database | tableName | isTemporary | + +-----------+------------+--------------+--+ + | default | sam | false | + | default | sam1 | false | + | default | suj | false | + +-----------+------------+--------------+--+ + +{% endhighlight %} + +### Related statements +- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) +- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index 7fcd397915825..e311691c6b801 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -19,4 +19,240 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`ALTER TABLE` statement changes the schema or properties of a table. + +### RENAME +`ALTER TABLE RENAME` statement changes the table name of an existing table in the database. + +#### Syntax +{% highlight sql %} +ALTER TABLE [db_name.]old_table_name RENAME TO [db_name.]new_table_name + +ALTER TABLE table_name PARTITION partition_spec RENAME TO PARTITION partition_spec; + +{% endhighlight %} + +#### Parameters +
+
old_table_name
+
Name of an existing table.
+
+
+
db_name
+
Name of the existing database.
+
+ +
+
new_table_name
+
New name using which the table has to be renamed.
+
+ +
+
partition_spec
+
Partition to be renamed.
+
+ + +### ADD COLUMNS +`ALTER TABLE ADD COLUMNS` statement adds mentioned columns to an existing table. + +#### Syntax +{% highlight sql %} +ALTER TABLE table_name ADD COLUMNS (col_spec[, col_spec ...]) +{% endhighlight %} + +#### Parameters +
+
table_name
+
The name of an existing table.
+
+ + +
+
COLUMNS (col_spec)
+
Specifies the columns to be added to be renamed.
+
+ + +### SET AND UNSET + +#### SET TABLE PROPERTIES +`ALTER TABLE SET` command is used for setting the table properties. If a particular property was already set, +this overrides the old value with the new one. + +`ALTER TABLE UNSET` is used to drop the table property. + +##### Syntax +{% highlight sql %} + +--Set Table Properties +ALTER TABLE table_name SET TBLPROPERTIES (key1=val1, key2=val2, ...) + +--Unset Table Properties +ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] (key1, key2, ...) + +{% endhighlight %} + +#### SET SERDE +`ALTER TABLE SET` command is used for setting the SERDE or SERDE properties in Hive tables. If a particular property was already set, +this overrides the old value with the new one. + +##### Syntax +{% highlight sql %} + +--Set SERDE Propeties +ALTER TABLE table_name [PARTITION part_spec] + SET SERDEPROPERTIES (key1=val1, key2=val2, ...) + +ALTER TABLE table_name [PARTITION part_spec] SET SERDE serde_class_name + [WITH SERDEPROPERTIES (key1=val1, key2=val2, ...)] + +{% endhighlight %} + +#### SET LOCATION And SET FILE FORMAT +`ALTER TABLE SET` command can also be used for changing the file location and file format for +exsisting tables. + +##### Syntax +{% highlight sql %} + +--Changing File Format +ALTER TABLE table_name [PARTITION partition_spec] SET FILEFORMAT file_format; + +--Changing File Location +ALTER TABLE table_name [PARTITION partition_spec] SET LOCATION 'new_location'; + +{% endhighlight %} + +#### Parameters +
+
table_name
+
The name of an existing table.
+
+ +
+
PARTITION (part_spec)
+
Specifies the partition on which the property has to be set.
+
+ +
+
SERDEPROPERTIES (key1=val1, key2=val2, ...)
+
Specifies the SERDE properties to be set.
+
+ + +### Examples +{% highlight sql %} + +--RENAME table +DESC student; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+--+ + +ALTER TABLE Student RENAME TO StudentInfo; + +--After Renaming the table + +DESC StudentInfo; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+--+ + +--RENAME partition + +SHOW PARTITIONS StudentInfo; ++------------+--+ +| partition | ++------------+--+ +| age=10 | +| age=11 | +| age=12 | ++------------+--+ + +ALTER TABLE default.StudentInfo PARTITION (age='10') RENAME TO PARTITION (age='15'); + +--After renaming Partition +SHOW PARTITIONS StudentInfo; ++------------+--+ +| partition | ++------------+--+ +| age=11 | +| age=12 | +| age=15 | ++------------+--+ + +-- Add new column to a table + +DESC StudentInfo; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+ + +ALTER TABLE StudentInfo ADD columns (LastName string, DOB timestamp); + +--After Adding New columns to the table +DESC StudentInfo; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| LastName | string | NULL | +| DOB | timestamp | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+--+ + + +--Change the fileformat +ALTER TABLE loc_orc SET fileformat orc; + +ALTER TABLE p1 partition (month=2, day=2) SET fileformat parquet; + +--Change the file Location +ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways' + +-- SET SERDE/ SERDE Properties +ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +ALTER TABLE dbx.tab1 SET SERDE 'org.apache.madoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee') + +--SET TABLE PROPERTIES +ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('winner' = 'loser') + +--DROP TABLE PROPERTIES +ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('winner') + +{% endhighlight %} + + +### Related Statements +- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + + diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md index f95a9eba42c2f..4c09ebafb1f5d 100644 --- a/docs/sql-ref-syntax-ddl-create-function.md +++ b/docs/sql-ref-syntax-ddl-create-function.md @@ -19,4 +19,153 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The `CREATE FUNCTION` statement is used to create a temporary or permanent function +in Spark. Temporary functions are scoped at a session level where as permanent +functions are created in the persistent catalog and are made available to +all sessions. The resources specified in the `USING` clause are made available +to all executors when they are executed for the first time. In addition to the +SQL interface, spark allows users to create custom user defined scalar and +aggregate functions using Scala, Python and Java APIs. Please refer to +[scalar_functions](sql-getting-started.html#scalar-functions) and +[aggregate functions](sql-getting-started#aggregations) for more information. + +### Syntax +{% highlight sql %} +CREATE [ OR REPLACE ] [ TEMPORARY ] FUNCTION [ IF NOT EXISTS ] + function_name AS class_name [ resource_locations ] +{% endhighlight %} + +### Parameters +
+
OR REPLACE
+
+ If specified, the resources for the function are reloaded. This is mainly useful + to pick up any changes made to the implementation of the function. This + parameter is mutually exclusive to IF NOT EXISTS and can not + be specified together. +
+
TEMPORARY
+
+ Indicates the scope of function being created. When TEMPORARY is specified, the + created function is valid and visible in the current session. No persistent + entry is made in the catalog for these kind of functions. +
+
IF NOT EXISTS
+
+ If specified, creates the function only when it does not exist. The creation + of function succeeds (no error is thrown) if the specified function already + exists in the system. This parameter is mutually exclusive to OR REPLACE + and can not be specified together. +
+
function_name
+
+ Specifies a name of funnction to be created. The function name may be + optionally qualified with a database name.

+ Syntax: + + [database_name.]function_name + +
+
class_name
+
+ Specifies the name of the class that provides the implementation for function to be created. + The implementing class should extend one of the base classes as follows: +
    +
  • Should extend UDF or UDAF in org.apache.hadoop.hive.ql.exec package.
  • +
  • Should extend AbstractGenericUDAFResolver, GenericUDF, or + GenericUDTF in org.apache.hadoop.hive.ql.udf.generic package.
  • +
  • Should extend UserDefinedAggregateFunction in org.apache.spark.sql.expressions package.
  • +
+
+
resource_locations
+
+ Specifies the list of resources that contain the implementation of the function + along with its dependencies.

+ Syntax: + + USING { { (JAR | FILE ) resource_uri} , ...} + +
+
+ +### Examples +{% highlight sql %} +-- 1. Create a simple UDF `SimpleUdf` that increments the supplied integral value by 10. +-- import org.apache.hadoop.hive.ql.exec.UDF; +-- public class SimpleUdf extends UDF { +-- public int evaluate(int value) { +-- return value + 10; +-- } +-- } +-- 2. Compile and place it in a JAR file called `SimpleUdf.jar` in /tmp. + +-- Create a table called `test` and insert two rows. +CREATE TABLE test(c1 INT); +INSERT INTO test VALUES (1), (2); + +-- Create a permanent function called `simple_udf`. +CREATE FUNCTION simple_udf AS 'SimpleUdf' + USING JAR '/tmp/SimpleUdf.jar'; + +-- Verify that the function is in the registry. +SHOW USER FUNCTIONS; + +------------------+ + | function| + +------------------+ + |default.simple_udf| + +------------------+ + +-- Invoke the function. Every selected value should be incremented by 10. +SELECT simple_udf(c1) AS function_return_value FROM t1; + +---------------------+ + |function_return_value| + +---------------------+ + | 11| + | 12| + +---------------------+ + +-- Created a temporary function. +CREATE TEMPORARY FUNCTION simple_temp_udf AS 'SimpleUdf' + USING JAR '/tmp/SimpleUdf.jar'; + +-- Verify that the newly created temporary function is in the registry. +-- Please note that the temporary function does not have a qualified +-- database associated with it. +SHOW USER FUNCTIONS; + +------------------+ + | function| + +------------------+ + |default.simple_udf| + | simple_temp_udf| + +------------------+ + +-- 1. Modify `SimpleUdf`'s implementation to add supplied integral value by 20. +-- import org.apache.hadoop.hive.ql.exec.UDF; + +-- public class SimpleUdfR extends UDF { +-- public int evaluate(int value) { +-- return value + 20; +-- } +-- } +-- 2. Compile and place it in a jar file called `SimpleUdfR.jar` in /tmp. + +-- Replace the implementation of `simple_udf` +CREATE OR REPLACE FUNCTION simple_udf AS 'SimpleUdfR' + USING JAR '/tmp/SimpleUdfR.jar'; + +-- Invoke the function. Every selected value should be incremented by 20. +SELECT simple_udf(c1) AS function_return_value FROM t1; ++---------------------+ +|function_return_value| ++---------------------+ +| 21| +| 22| ++---------------------+ + +{% endhighlight %} + +### Related statements +- [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) +- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) +- [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) diff --git a/docs/sql-ref-syntax-ddl-drop-function.md b/docs/sql-ref-syntax-ddl-drop-function.md index 9cf1f445e91f4..f1b54c675cef5 100644 --- a/docs/sql-ref-syntax-ddl-drop-function.md +++ b/docs/sql-ref-syntax-ddl-drop-function.md @@ -19,4 +19,87 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The `DROP FUNCTION` statement drops a temporary or user defined function (UDF). An exception will + be thrown if the function does not exist. + +### Syntax +{% highlight sql %} +DROP [TEMPORARY] FUNCTION [IF EXISTS] [db_name.]function_name; +{% endhighlight %} + + +### Parameters + +
+
function_name
+
The name of an existing function.
+
+ +
+
TEMPORARY
+
Should be used to delete the `temporary` function.
+
+ +
+
IF EXISTS
+
If specified, no exception is thrown when the function does not exist.
+
+ +### Example +{% highlight sql %} +-- Create a permanent function `test_avg` +CREATE FUNCTION test_avg as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'; + +-- List user functions +SHOW USER FUNCTIONS; + +-------------------+ + | function | + +-------------------+ + | default.test_avg | + +-------------------+ + +-- Create Temporary function `test_avg` +CREATE TEMPORARY FUNCTION test_avg as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'; + +-- List user functions +SHOW USER FUNCTIONS; + +-------------------+ + | function | + +-------------------+ + | default.test_avg | + | test_avg | + +-------------------+ + +-- Drop Permanent function +DROP FUNCTION test_avg; + +---------+ + | Result | + +---------+ + +---------+ + +-- Try to drop Permanent function which is not present +DROP FUNCTION test_avg; + Error: Error running query: + org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException: + Function 'default.test_avg' not found in database 'default'; (state=,code=0) + +-- List the functions after dropping, it should list only temporary function +SHOW USER FUNCTIONS; + +-----------+ + | function | + +-----------+ + | test_avg | + +-----------+ + +-- Drop Temporary function +DROP TEMPORARY FUNCTION IF EXISTS test_avg; + +---------+ + | Result | + +---------+ + +---------+ +{% endhighlight %} +### Related statements +- [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) +- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) +- [SHOW FUNCTION](sql-ref-syntax-aux-show-functions.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-ddl-repair-table.md b/docs/sql-ref-syntax-ddl-repair-table.md index f21de558b8a07..2816d0bba3c9f 100644 --- a/docs/sql-ref-syntax-ddl-repair-table.md +++ b/docs/sql-ref-syntax-ddl-repair-table.md @@ -19,4 +19,45 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`MSCK REPAIR TABLE` recovers all the partitions in the directory of a table and updates the Hive metastore. When creating a table using `PARTITIONED BY` clause, partitions are generated and registered in the Hive metastore. However, if the partitioned table is created from existing data, partitions are not registered automatically in the Hive metastore. User needs to run `MSCK REPAIR TABLE` to register the partitions. `MSCK REPAIR TABLE` on a non-existent table or a table without partitions throws an exception. Another way to recover partitions is to use `ALTER TABLE RECOVER PARTITIONS`. + +### Syntax +{% highlight sql %} +MSCK REPAIR TABLE table_name +{% endhighlight %} + +### Parameters +
+
table_name
+
Specifies the name of the table to be repaired.
+
+ +### Examples +{% highlight sql %} + -- create a partitioned table from existing data /tmp/namesAndAges.parquet + CREATE TABLE t1 (name STRING, age INT) USING parquet PARTITIONED BY (age) + location "/tmp/namesAndAges.parquet"; + + -- SELECT * FROM t1 does not return results + SELECT * FROM t1; + + -- run MSCK REPAIR TABLE to recovers all the partitions + MSCK REPAIR TABLE t1; + + -- SELECT * FROM t1 returns results + SELECT * FROM t1; + + + -------------- + ------+ + | name | age | + + -------------- + ------+ + | Michael | 20 | + + -------------- + ------+ + | Justin | 19 | + + -------------- + ----- + + | Andy | 30 | + + -------------- + ----- + + +{% endhighlight %} +### Related Statements + * [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) diff --git a/docs/sql-ref-syntax-dml-load.md b/docs/sql-ref-syntax-dml-load.md index fd25ba314e0b6..c2a6102db4aad 100644 --- a/docs/sql-ref-syntax-dml-load.md +++ b/docs/sql-ref-syntax-dml-load.md @@ -1,7 +1,7 @@ --- layout: global -title: LOAD -displayTitle: LOAD +title: LOAD DATA +displayTitle: LOAD DATA license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,4 +19,101 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`LOAD DATA` statement loads the data into a table from the user specified directory or file. If a directory is specified then all the files from the directory are loaded. If a file is specified then only the single file is loaded. Additionally the `LOAD DATA` statement takes an optional partition specification. When a partition is specified, the data files (when input source is a directory) or the single file (when input source is a file) are loaded into the partition of the target table. + +### Syntax +{% highlight sql %} +LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_name + [ PARTITION ( partition_col_name = partition_col_val [ , ... ] ) ] +{% endhighlight %} + +### Parameters +
+
path
+
Path of the file system. It can be either an absolute or a relative path.
+
+ +
+
table_name
+
The name of an existing table.
+
+ +
+
PARTITION ( partition_col_name = partition_col_val [ , ... ] )
+
Specifies one or more partition column and value pairs.
+
+ +
+
LOCAL
+
If specified, it causes the INPATH to be resolved against the local file system, instead of the default file system, which is typically a distributed storage.
+
+ +
+
OVERWRITE
+
By default, new data is appended to the table. If OVERWRITE is used, the table is instead overwritten with new data.
+
+ +### Examples +{% highlight sql %} + -- Example without partition specification. + -- Assuming the students table has already been created and populated. + SELECT * FROM students; + + + -------------- + ------------------------------ + -------------- + + | name | address | student_id | + + -------------- + ------------------------------ + -------------- + + | Amy Smith | 123 Park Ave, San Jose | 111111 | + + -------------- + ------------------------------ + -------------- + + + CREATE TABLE test_load (name VARCHAR(64), address VARCHAR(64), student_id INT); + + -- Assuming the students table is in '/user/hive/warehouse/' + LOAD DATA LOCAL INPATH '/user/hive/warehouse/students' OVERWRITE INTO TABLE test_load; + + SELECT * FROM test_load; + + + -------------- + ------------------------------ + -------------- + + | name | address | student_id | + + -------------- + ------------------------------ + -------------- + + | Amy Smith | 123 Park Ave, San Jose | 111111 | + + -------------- + ------------------------------ + -------------- + + + -- Example with partition specification. + CREATE TABLE test_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); + + INSERT INTO test_partition PARTITION (c2 = 2, c3 = 3) VALUES (1); + + INSERT INTO test_partition PARTITION (c2 = 5, c3 = 6) VALUES (4); + + INSERT INTO test_partition PARTITION (c2 = 8, c3 = 9) VALUES (7); + + SELECT * FROM test_partition; + + + ------- + ------- + ----- + + | c1 | c2 | c3 | + + ------- + --------------- + + | 1 | 2 | 3 | + + ------- + ------- + ----- + + | 4 | 5 | 6 | + + ------- + ------- + ----- + + | 7 | 8 | 9 | + + ------- + ------- + ----- + + + CREATE TABLE test_load_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); + + -- Assuming the test_partition table is in '/user/hive/warehouse/' + LOAD DATA LOCAL INPATH '/user/hive/warehouse/test_partition/c2=2/c3=3' + OVERWRITE INTO TABLE test_load_partition PARTITION (c2=2, c3=3); + + SELECT * FROM test_load_partition; + + + ------- + ------- + ----- + + | c1 | c2 | c3 | + + ------- + --------------- + + | 1 | 2 | 3 | + + ------- + ------- + ----- + + + +{% endhighlight %} + diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index ef4cdc2608f45..89732d309aa27 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -362,6 +362,27 @@ The following configurations are optional: + + + + + + + + + + + + + + - - - - - - - - - - - {errorMessageCell(detail)} - - } - Some(UIUtils.listingTable(headerRow, generateDataRow, - dataRows, false, None, Seq(null), false)) + val parameterSqlTablePage = request.getParameter(s"$sqlTableTag.page") + val parameterSqlTableSortColumn = request.getParameter(s"$sqlTableTag.sort") + val parameterSqlTableSortDesc = request.getParameter(s"$sqlTableTag.desc") + val parameterSqlPageSize = request.getParameter(s"$sqlTableTag.pageSize") + + val sqlTablePage = Option(parameterSqlTablePage).map(_.toInt).getOrElse(1) + val sqlTableSortColumn = Option(parameterSqlTableSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Start Time") + val sqlTableSortDesc = Option(parameterSqlTableSortDesc).map(_.toBoolean).getOrElse( + // New executions should be shown above old executions by default. + sqlTableSortColumn == "Start Time" + ) + val sqlTablePageSize = Option(parameterSqlPageSize).map(_.toInt).getOrElse(100) + + try { + Some(new SqlStatsPagedTable( + request, + parent, + listener.getExecutionList, + "sqlserver", + UIUtils.prependBaseUri(request, parent.basePath), + parameterOtherTable, + sqlTableTag, + pageSize = sqlTablePageSize, + sortColumn = sqlTableSortColumn, + desc = sqlTableSortDesc + ).table(sqlTablePage)) + } catch { + case e@(_: IllegalArgumentException | _: IndexOutOfBoundsException) => + Some(
+

Error while rendering job table:

+
+              {Utils.exceptionString(e)}
+            
+
) + } } else { None } @@ -118,30 +137,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" content } - private def errorMessageCell(errorMessage: String): Seq[Node] = { - val isMultiline = errorMessage.indexOf('\n') >= 0 - val errorSummary = StringEscapeUtils.escapeHtml4( - if (isMultiline) { - errorMessage.substring(0, errorMessage.indexOf('\n')) - } else { - errorMessage - }) - val details = if (isMultiline) { - // scalastyle:off - - + details - ++ - - // scalastyle:on - } else { - "" - } - - } - /** Generate stats of batch sessions of the thrift server program */ private def generateSessionStatsTable(request: HttpServletRequest): Seq[Node] = { val sessionList = listener.getSessionList @@ -180,7 +175,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" content } - /** * Returns a human-readable string representing a duration such as "5 second 35 ms" */ @@ -197,3 +191,254 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" } } +private[ui] class SqlStatsPagedTable( + request: HttpServletRequest, + parent: ThriftServerTab, + data: Seq[ExecutionInfo], + subPath: String, + basePath: String, + parameterOtherTable: Iterable[String], + sqlStatsTableTag: String, + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedTable[SqlStatsTableRow] { + + override val dataSource = new SqlStatsTableDataSource(data, pageSize, sortColumn, desc) + + private val parameterPath = s"$basePath/$subPath/?${parameterOtherTable.mkString("&")}" + + override def tableId: String = sqlStatsTableTag + + 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.name()) + parameterPath + + s"&$pageNumberFormField=$page" + + s"&$sqlStatsTableTag.sort=$encodedSortColumn" + + s"&$sqlStatsTableTag.desc=$desc" + + s"&$pageSizeFormField=$pageSize" + } + + override def pageSizeFormField: String = s"$sqlStatsTableTag.pageSize" + + override def pageNumberFormField: String = s"$sqlStatsTableTag.page" + + override def goButtonFormPath: String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + s"$parameterPath&$sqlStatsTableTag.sort=$encodedSortColumn&$sqlStatsTableTag.desc=$desc" + } + + override def headers: Seq[Node] = { + val sqlTableHeaders = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", + "Close Time", "Execution Time", "Duration", "Statement", "State", "Detail") + + val tooltips = Seq(None, None, None, None, Some(THRIFT_SERVER_FINISH_TIME), + Some(THRIFT_SERVER_CLOSE_TIME), Some(THRIFT_SERVER_EXECUTION), + Some(THRIFT_SERVER_DURATION), None, None, None) + + assert(sqlTableHeaders.length == tooltips.length) + + val headerRow: Seq[Node] = { + sqlTableHeaders.zip(tooltips).map { case (header, tooltip) => + if (header == sortColumn) { + val headerLink = Unparsed( + parameterPath + + s"&$sqlStatsTableTag.sort=${URLEncoder.encode(header, UTF_8.name())}" + + s"&$sqlStatsTableTag.desc=${!desc}" + + s"&$sqlStatsTableTag.pageSize=$pageSize" + + s"#$sqlStatsTableTag") + val arrow = if (desc) "▾" else "▴" // UP or DOWN + + if (tooltip.nonEmpty) { + + } else { + + } + } else { + val headerLink = Unparsed( + parameterPath + + s"&$sqlStatsTableTag.sort=${URLEncoder.encode(header, UTF_8.name())}" + + s"&$sqlStatsTableTag.pageSize=$pageSize" + + s"#$sqlStatsTableTag") + + if(tooltip.nonEmpty) { + + } else { + + } + } + } + } + + {headerRow} + + } + + override def row(sqlStatsTableRow: SqlStatsTableRow): Seq[Node] = { + val info = sqlStatsTableRow.executionInfo + val startTime = info.startTimestamp + val executionTime = sqlStatsTableRow.executionTime + val duration = sqlStatsTableRow.duration + + def jobLinks(jobData: Seq[String]): Seq[Node] = { + jobData.map { jobId => + [{jobId.toString}] + } + } + + + + + + + + + + + + + {errorMessageCell(sqlStatsTableRow.detail)} + + } + + + private def errorMessageCell(errorMessage: String): Seq[Node] = { + val isMultiline = errorMessage.indexOf('\n') >= 0 + val errorSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + errorMessage.substring(0, errorMessage.indexOf('\n')) + } else { + errorMessage + }) + val details = if (isMultiline) { + // scalastyle:off + + + details + ++ + + // scalastyle:on + } else { + "" + } + + } + + private def jobURL(request: HttpServletRequest, jobId: String): String = + "%s/jobs/job/?id=%s".format(UIUtils.prependBaseUri(request, parent.basePath), jobId) +} + + private[ui] class SqlStatsTableRow( + val jobId: Seq[String], + val duration: Long, + val executionTime: Long, + val executionInfo: ExecutionInfo, + val detail: String) + + private[ui] class SqlStatsTableDataSource( + info: Seq[ExecutionInfo], + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedDataSource[SqlStatsTableRow](pageSize) { + + // Convert ExecutionInfo to SqlStatsTableRow 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 = info.map(sqlStatsTableRow).sorted(ordering(sortColumn, desc)) + + private var _slicedStartTime: Set[Long] = null + + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[SqlStatsTableRow] = { + val r = data.slice(from, to) + r.map(x => x) + _slicedStartTime = r.map(_.executionInfo.startTimestamp).toSet + r + } + + private def sqlStatsTableRow(executionInfo: ExecutionInfo): SqlStatsTableRow = { + val duration = executionInfo.totalTime(executionInfo.closeTimestamp) + val executionTime = executionInfo.totalTime(executionInfo.finishTimestamp) + val detail = Option(executionInfo.detail).filter(!_.isEmpty) + .getOrElse(executionInfo.executePlan) + val jobId = executionInfo.jobId.toSeq.sorted + + new SqlStatsTableRow(jobId, duration, executionTime, executionInfo, detail) + + } + + /** + * Return Ordering according to sortColumn and desc. + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[SqlStatsTableRow] = { + val ordering: Ordering[SqlStatsTableRow] = sortColumn match { + case "User" => Ordering.by(_.executionInfo.userName) + case "JobID" => Ordering by (_.jobId.headOption) + case "GroupID" => Ordering.by(_.executionInfo.groupId) + case "Start Time" => Ordering.by(_.executionInfo.startTimestamp) + case "Finish Time" => Ordering.by(_.executionInfo.finishTimestamp) + case "Close Time" => Ordering.by(_.executionInfo.closeTimestamp) + case "Execution Time" => Ordering.by(_.executionTime) + case "Duration" => Ordering.by(_.duration) + case "Statement" => Ordering.by(_.executionInfo.statement) + case "State" => Ordering.by(_.executionInfo.state) + case "Detail" => Ordering.by(_.detail) + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") + } + if (desc) { + ordering.reverse + } else { + ordering + } + } + + } 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 b752634aef956..0aa0a2b8335d8 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 @@ -26,6 +26,7 @@ import org.apache.commons.text.StringEscapeUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState} +import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ @@ -81,6 +82,10 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) val table = if (numStatement > 0) { val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", "Execution Time", "Duration", "Statement", "State", "Detail") + val tooltips = Seq(None, None, None, None, Some(THRIFT_SERVER_FINISH_TIME), + Some(THRIFT_SERVER_CLOSE_TIME), Some(THRIFT_SERVER_EXECUTION), + Some(THRIFT_SERVER_DURATION), None, None, None) + assert(headerRow.length == tooltips.length) val dataRows = executionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -113,7 +118,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) } Some(UIUtils.listingTable(headerRow, generateDataRow, - dataRows, false, None, Seq(null), false)) + dataRows, false, None, Seq(null), false, tooltipHeaders = tooltips)) } else { None } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index db2066009b351..8efb2c3311cfe 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -39,7 +39,7 @@ private[thriftserver] class ThriftServerTab(sparkContext: SparkContext) attachPage(new ThriftServerSessionPage(this)) parent.attachTab(this) - def detach() { + def detach(): Unit = { getSparkUI(sparkContext).detachTab(this) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala similarity index 65% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala rename to sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala index 035bc09d31871..1990b8f2d3285 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala @@ -15,13 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.plans.logical.sql +package org.apache.spark.sql.hive.thriftserver.ui -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +private[ui] object ToolTips { + val THRIFT_SERVER_FINISH_TIME = + "Execution finish time, before fetching the results" -case class DeleteFromStatement( - tableName: Seq[String], - tableAlias: Option[String], - condition: Option[Expression]) - extends ParsedStatement + val THRIFT_SERVER_CLOSE_TIME = + "Operation close time after fetching the results" + + val THRIFT_SERVER_EXECUTION = + "Difference between start time and finish time" + + val THRIFT_SERVER_DURATION = + "Difference between start time and close time" +} 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 6d45041e12821..04e7f579ff712 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -27,12 +27,11 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.contrib.udf.example.UDFExampleFormat 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.hive.test.HiveTestJars import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -202,7 +201,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } test("Commands using SerDe provided in --jars") { - val jarFile = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath + val jarFile = HiveTestJars.getHiveHcatalogCoreJar().getCanonicalPath val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") @@ -227,6 +226,32 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { ) } + test("SPARK-29022: Commands using SerDe provided in --hive.aux.jars.path") { + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + val hiveContribJar = HiveTestJars.getHiveHcatalogCoreJar().getCanonicalPath + runCliWithin( + 3.minute, + Seq("--conf", s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( + """CREATE TABLE addJarWithHiveAux(key string, val string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; + """.stripMargin + -> "", + "CREATE TABLE sourceTableForWithHiveAux (key INT, val STRING);" + -> "", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTableForWithHiveAux;" + -> "", + "INSERT INTO TABLE addJarWithHiveAux SELECT key, val FROM sourceTableForWithHiveAux;" + -> "", + "SELECT collect_list(array(val)) FROM addJarWithHiveAux;" + -> """[["val_238"],["val_86"],["val_311"],["val_27"],["val_165"]]""", + "DROP TABLE addJarWithHiveAux;" + -> "", + "DROP TABLE sourceTableForWithHiveAux;" + -> "" + ) + } + test("SPARK-11188 Analysis error reporting") { runCliWithin(timeout = 2.minute, errorResponses = Seq("AnalysisException"))( @@ -297,11 +322,12 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } test("Support hive.aux.jars.path") { - val hiveContribJar = HiveTestUtils.getHiveContribJar.getCanonicalPath + val hiveContribJar = HiveTestJars.getHiveContribJar().getCanonicalPath runCliWithin( 1.minute, Seq("--conf", s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( - s"CREATE TEMPORARY FUNCTION example_format AS '${classOf[UDFExampleFormat].getName}';" -> "", + "CREATE TEMPORARY FUNCTION example_format AS " + + "'org.apache.hadoop.hive.contrib.udf.example.UDFExampleFormat';" -> "", "SELECT example_format('%o', 93);" -> "135" ) } @@ -319,7 +345,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { test("SPARK-28840 test --jars and hive.aux.jars.path command") { val jarFile = new File("../../sql/hive/src/test/resources/SPARK-21101-1.0.jar").getCanonicalPath - val hiveContribJar = HiveTestUtils.getHiveContribJar.getCanonicalPath + val hiveContribJar = HiveTestJars.getHiveContribJar().getCanonicalPath runCliWithin( 1.minute, Seq("--jars", s"$jarFile", "--conf", @@ -332,4 +358,39 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "SELECT concat_ws(',', 'First', example_max(1234321), 'Third');" -> "First,1234321,Third" ) } + + test("SPARK-29022 Commands using SerDe provided in ADD JAR sql") { + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + val hiveContribJar = HiveTestJars.getHiveHcatalogCoreJar().getCanonicalPath + runCliWithin( + 3.minute)( + s"ADD JAR ${hiveContribJar};" -> "", + """CREATE TABLE addJarWithSQL(key string, val string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; + """.stripMargin + -> "", + "CREATE TABLE sourceTableForWithSQL(key INT, val STRING);" + -> "", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTableForWithSQL;" + -> "", + "INSERT INTO TABLE addJarWithSQL SELECT key, val FROM sourceTableForWithSQL;" + -> "", + "SELECT collect_list(array(val)) FROM addJarWithSQL;" + -> """[["val_238"],["val_86"],["val_311"],["val_27"],["val_165"]]""", + "DROP TABLE addJarWithSQL;" + -> "", + "DROP TABLE sourceTableForWithSQL;" + -> "" + ) + } + + test("SPARK-26321 Should not split semicolon within quoted string literals") { + runCliWithin(3.minute)( + """select 'Test1', "^;^";""" -> "Test1\t^;^", + """select 'Test2', "\";";""" -> "Test2\t\";", + """select 'Test3', "\';";""" -> "Test3\t';", + "select concat('Test4', ';');" -> "Test4;" + ) + } } 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 3fef23f8eed3e..3c8d25d935ec0 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 @@ -23,6 +23,7 @@ import java.nio.charset.StandardCharsets import java.sql.{Date, DriverManager, SQLException, Statement} import java.util.{Locale, UUID} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future, Promise} @@ -34,7 +35,7 @@ import com.google.common.io.Files 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.{FetchOrientation, FetchType, GetInfoType, RowSet} import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket @@ -43,7 +44,7 @@ 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.hive.test.HiveTestJars 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} @@ -492,7 +493,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { withMultipleConnectionJdbcStatement("smallKV", "addJar")( { statement => - val jarFile = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath + val jarFile = HiveTestJars.getHiveHcatalogCoreJar().getCanonicalPath statement.executeQuery(s"ADD JAR $jarFile") }, @@ -684,6 +685,92 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { assert(e.getMessage.contains("org.apache.spark.sql.catalyst.parser.ParseException")) } } + + test("ThriftCLIService FetchResults FETCH_FIRST, FETCH_NEXT, FETCH_PRIOR") { + def checkResult(rows: RowSet, start: Long, end: Long): Unit = { + assert(rows.getStartOffset() == start) + assert(rows.numRows() == end - start) + rows.iterator.asScala.zip((start until end).iterator).foreach { case (row, v) => + assert(row(0).asInstanceOf[Long] === v) + } + } + + withCLIServiceClient { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val operationHandle = client.executeStatement( + sessionHandle, + "SELECT * FROM range(10)", + confOverlay) // 10 rows result with sequence 0, 1, 2, ..., 9 + var rows: RowSet = null + + // Fetch 5 rows with FETCH_NEXT + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 5, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 5) // fetched [0, 5) + + // Fetch another 2 rows with FETCH_NEXT + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 2, FetchType.QUERY_OUTPUT) + checkResult(rows, 5, 7) // fetched [5, 7) + + // FETCH_PRIOR 3 rows + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 3, FetchType.QUERY_OUTPUT) + checkResult(rows, 2, 5) // fetched [2, 5) + + // FETCH_PRIOR again will scroll back to 0, and then the returned result + // may overlap the results of previous FETCH_PRIOR + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 3, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 3) // fetched [0, 3) + + // FETCH_PRIOR again will stay at 0 + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 4, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 4) // fetched [0, 4) + + // FETCH_NEXT will continue moving forward from offset 4 + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 10, FetchType.QUERY_OUTPUT) + checkResult(rows, 4, 10) // fetched [4, 10) until the end of results + + // FETCH_NEXT is at end of results + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 5, FetchType.QUERY_OUTPUT) + checkResult(rows, 10, 10) // fetched empty [10, 10) (at end of results) + + // FETCH_NEXT is at end of results again + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 2, FetchType.QUERY_OUTPUT) + checkResult(rows, 10, 10) // fetched empty [10, 10) (at end of results) + + // FETCH_PRIOR 1 rows yet again + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 1, FetchType.QUERY_OUTPUT) + checkResult(rows, 9, 10) // fetched [9, 10) + + // FETCH_NEXT will return 0 yet again + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 5, FetchType.QUERY_OUTPUT) + checkResult(rows, 10, 10) // fetched empty [10, 10) (at end of results) + + // FETCH_FIRST results from first row + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_FIRST, 3, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 3) // fetch [0, 3) + + // Fetch till the end rows with FETCH_NEXT" + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 1000, FetchType.QUERY_OUTPUT) + checkResult(rows, 3, 10) // fetched [3, 10) + + client.closeOperation(operationHandle) + client.closeSession(sessionHandle) + } + } } class SingleSessionSuite extends HiveThriftJdbcTest { @@ -842,7 +929,7 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { s"jdbc:hive2://localhost:$serverPort/?${hiveConfList}#${hiveVarList}" } - def withMultipleConnectionJdbcStatement(tableNames: String*)(fs: (Statement => Unit)*) { + def withMultipleConnectionJdbcStatement(tableNames: String*)(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } val statements = connections.map(_.createStatement()) @@ -863,7 +950,7 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { } } - def withDatabase(dbNames: String*)(fs: (Statement => Unit)*) { + def withDatabase(dbNames: String*)(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } val statements = connections.map(_.createStatement()) @@ -879,7 +966,7 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { } } - def withJdbcStatement(tableNames: String*)(f: Statement => Unit) { + def withJdbcStatement(tableNames: String*)(f: Statement => Unit): Unit = { withMultipleConnectionJdbcStatement(tableNames: _*)(f) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 381b8f2324ca6..04b1de00ccbff 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -19,18 +19,20 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.sql.{DriverManager, SQLException, Statement, Timestamp} -import java.util.Locale +import java.util.{Locale, MissingFormatArgumentException} import scala.util.{Random, Try} import scala.util.control.NonFatal +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.HiveSQLException -import org.scalatest.Ignore -import org.apache.spark.sql.{AnalysisException, SQLQueryTestSuite} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql.SQLQueryTestSuite +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.util.fileToString import org.apache.spark.sql.execution.HiveResult +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -43,12 +45,12 @@ import org.apache.spark.sql.types._ * 2. Support DESC command. * 3. Support SHOW command. */ -@Ignore class ThriftServerQueryTestSuite extends SQLQueryTestSuite { private var hiveServer2: HiveThriftServer2 = _ - override def beforeEach(): Unit = { + override def beforeAll(): Unit = { + super.beforeAll() // Chooses a random port between 10000 and 19999 var listeningPort = 10000 + Random.nextInt(10000) @@ -65,36 +67,34 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { logInfo("HiveThriftServer2 started successfully") } - override def afterEach(): Unit = { - hiveServer2.stop() + override def afterAll(): Unit = { + try { + hiveServer2.stop() + } finally { + super.afterAll() + } } override val isTestWithConfigSets = false /** List of test cases to ignore, in lower cases. */ - override def blackList: Set[String] = Set( - "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. + override def blackList: Set[String] = super.blackList ++ Set( // Missing UDF - "pgSQL/boolean.sql", - "pgSQL/case.sql", + "postgreSQL/boolean.sql", + "postgreSQL/case.sql", // SPARK-28624 "date.sql", - // SPARK-28619 - "pgSQL/aggregates_part1.sql", - "group-by.sql", // SPARK-28620 - "pgSQL/float4.sql", + "postgreSQL/float4.sql", // SPARK-28636 "decimalArithmeticOperations.sql", "literals.sql", "subquery/scalar-subquery/scalar-subquery-predicate.sql", "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-group-by.sql", "subquery/in-subquery/simple-in.sql", "subquery/in-subquery/in-order-by.sql", - "subquery/in-subquery/in-set-operations.sql", - // SPARK-28637 - "cast.sql", - "ansi/interval.sql" + "subquery/in-subquery/in-set-operations.sql" ) override def runQueries( @@ -111,13 +111,13 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { // PostgreSQL enabled cartesian product by default. statement.execute(s"SET ${SQLConf.CROSS_JOINS_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") - statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") + statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") case _ => } // Run the SQL queries preparing them for comparison. val outputs: Seq[QueryOutput] = queries.map { sql => - val output = getNormalizedResult(statement, sql) + val (_, output) = handleExceptions(getNormalizedResult(statement, sql)) // We might need to do some query canonicalization in the future. QueryOutput( sql = sql, @@ -136,8 +136,9 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "Try regenerate the result files.") Seq.tabulate(outputs.size) { i => val sql = segments(i * 3 + 1).trim + val schema = segments(i * 3 + 2).trim val originalOut = segments(i * 3 + 3) - val output = if (isNeedSort(sql)) { + val output = if (schema != emptySchema && isNeedSort(sql)) { originalOut.split("\n").sorted.mkString("\n") } else { originalOut @@ -166,19 +167,48 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { || d.sql.toUpperCase(Locale.ROOT).startsWith("DESC\n") || d.sql.toUpperCase(Locale.ROOT).startsWith("DESCRIBE ") || d.sql.toUpperCase(Locale.ROOT).startsWith("DESCRIBE\n") => + // Skip show command, see HiveResult.hiveResultString case s if s.sql.toUpperCase(Locale.ROOT).startsWith("SHOW ") || s.sql.toUpperCase(Locale.ROOT).startsWith("SHOW\n") => - // AnalysisException should exactly match. + + case _ if output.output.startsWith(classOf[NoSuchTableException].getPackage.getName) => + assert(expected.output.startsWith(classOf[NoSuchTableException].getPackage.getName), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + + case _ if output.output.startsWith(classOf[SparkException].getName) && + output.output.contains("overflow") => + assert(expected.output.contains(classOf[ArithmeticException].getName) && + expected.output.contains("overflow"), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + + case _ if output.output.startsWith(classOf[RuntimeException].getName) => + assert(expected.output.contains("Exception"), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + + case _ if output.output.startsWith(classOf[ArithmeticException].getName) && + output.output.contains("causes overflow") => + assert(expected.output.contains(classOf[ArithmeticException].getName) && + expected.output.contains("causes overflow"), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + + case _ if output.output.startsWith(classOf[MissingFormatArgumentException].getName) && + output.output.contains("Format specifier") => + assert(expected.output.contains(classOf[MissingFormatArgumentException].getName) && + expected.output.contains("Format specifier"), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + // SQLException should not exactly match. We only assert the result contains Exception. case _ if output.output.startsWith(classOf[SQLException].getName) => assert(expected.output.contains("Exception"), s"Exception did not match for query #$i\n${expected.sql}, " + s"expected: ${expected.output}, but got: ${output.output}") - // HiveSQLException is usually a feature that our ThriftServer cannot support. - // Please add SQL to blackList. - case _ if output.output.startsWith(classOf[HiveSQLException].getName) => - assert(false, s"${output.output} for query #$i\n${expected.sql}") + case _ => assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { output.output @@ -209,7 +239,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { Seq.empty - } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil } else { RegularTestCase(testCaseName, absPath, resultFile) :: Nil @@ -225,31 +255,30 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - private def getNormalizedResult(statement: Statement, sql: String): Seq[String] = { - try { - val rs = statement.executeQuery(sql) - val cols = rs.getMetaData.getColumnCount - val buildStr = () => (for (i <- 1 to cols) yield { - getHiveResult(rs.getObject(i)) - }).mkString("\t") - - val answer = Iterator.continually(rs.next()).takeWhile(identity).map(_ => buildStr()).toSeq - .map(replaceNotIncludedMsg) - if (isNeedSort(sql)) { - answer.sorted - } else { - answer + /** ThriftServer wraps the root exception, so it needs to be extracted. */ + override def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { + super.handleExceptions { + try { + result + } catch { + case NonFatal(e) => throw ExceptionUtils.getRootCause(e) } - } catch { - case a: AnalysisException => - // Do not output the logical plan tree which contains expression IDs. - // Also implement a crude way of masking expression IDs in the error message - // with a generic pattern "###". - val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage - Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x")).sorted - case NonFatal(e) => - // If there is an exception, put the exception class followed by the message. - Seq(e.getClass.getName, e.getMessage) + } + } + + private def getNormalizedResult(statement: Statement, sql: String): (String, Seq[String]) = { + val rs = statement.executeQuery(sql) + val cols = rs.getMetaData.getColumnCount + val buildStr = () => (for (i <- 1 to cols) yield { + getHiveResult(rs.getObject(i)) + }).mkString("\t") + + val answer = Iterator.continually(rs.next()).takeWhile(identity).map(_ => buildStr()).toSeq + .map(replaceNotIncludedMsg) + if (isNeedSort(sql)) { + ("", answer.sorted) + } else { + ("", answer) } } @@ -260,7 +289,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) } - private def withJdbcStatement(fs: (Statement => Unit)*) { + private def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") val serverPort = hiveServer2.getHiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname) @@ -337,7 +366,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { upperCase.startsWith("SELECT ") || upperCase.startsWith("SELECT\n") || upperCase.startsWith("WITH ") || upperCase.startsWith("WITH\n") || upperCase.startsWith("VALUES ") || upperCase.startsWith("VALUES\n") || - // pgSQL/union.sql + // postgreSQL/union.sql upperCase.startsWith("(") } 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 47cf4f104d204..7f731f3d05e51 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 @@ -24,8 +24,8 @@ import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest.{BeforeAndAfterAll, Matchers} import org.scalatest.concurrent.Eventually._ -import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ +import org.scalatestplus.selenium.WebBrowser import org.apache.spark.ui.SparkUICssErrorHandler diff --git a/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift index 7cd6fa37cec37..225e319737811 100644 --- a/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift +++ b/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift @@ -1028,7 +1028,6 @@ enum TFetchOrientation { FETCH_NEXT, // Get the previous rowset. The fetch offset is ignored. - // NOT SUPPORTED FETCH_PRIOR, // Return the rowset at the given fetch offset relative @@ -1056,8 +1055,8 @@ 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. + // The fetch orientation. This must be either + // FETCH_NEXT, FETCH_PRIOR or FETCH_FIRST. Defaults to FETCH_NEXT. 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT // Max number of rows that should be returned in diff --git a/sql/hive-thriftserver/v1.2.1/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 index 19153b654b08a..51bb28748d9e2 100644 --- a/sql/hive-thriftserver/v1.2.1/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 @@ -58,7 +58,10 @@ public abstract class Operation { private long lastAccessTime; protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = - EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); + EnumSet.of( + FetchOrientation.FETCH_NEXT, + FetchOrientation.FETCH_FIRST, + FetchOrientation.FETCH_PRIOR); protected Operation(HiveSession parentSession, OperationType opType, boolean runInBackground) { this.parentSession = parentSession; diff --git a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift b/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift index 824b04919073a..9026cd25df5b3 100644 --- a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift +++ b/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift @@ -1105,7 +1105,6 @@ enum TFetchOrientation { FETCH_NEXT, // Get the previous rowset. The fetch offset is ignored. - // NOT SUPPORTED FETCH_PRIOR, // Return the rowset at the given fetch offset relative @@ -1133,8 +1132,8 @@ 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. + // The fetch orientation. This must be either + // FETCH_NEXT, FETCH_PRIOR or FETCH_FIRST. Defaults to FETCH_NEXT. 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT // Max number of rows that should be returned in 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 index 788fcdee282ae..f26c715add987 100644 --- 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 @@ -64,7 +64,10 @@ public abstract class Operation { protected final QueryState queryState; protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = - EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); + EnumSet.of( + FetchOrientation.FETCH_NEXT, + FetchOrientation.FETCH_FIRST, + FetchOrientation.FETCH_PRIOR); protected Operation(HiveSession parentSession, OperationType opType) { this(parentSession, null, opType); diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index e7ff3a5f4be2b..36c19c680d165 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy /** * Runs the test cases that are included in the hive distribution. @@ -46,7 +47,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) } - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) @@ -59,13 +60,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) + // Ensures that the table insertion behaivor is consistent with Hive + TestHive.setConf(SQLConf.STORE_ASSIGNMENT_POLICY, StoreAssignmentPolicy.LEGACY.toString) // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") RuleExecutor.resetMetrics() } - override def afterAll() { + override def afterAll(): Unit = { try { TestHive.setCacheTables(false) TimeZone.setDefault(originalTimeZone) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index c7d953a731b9b..b0cf25c3a7813 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -37,7 +37,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte private val originalLocale = Locale.getDefault private val testTempDir = Utils.createTempDir() - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) @@ -100,7 +100,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte sql("set mapreduce.jobtracker.address=local") } - override def afterAll() { + override def afterAll(): Unit = { try { TestHive.setCacheTables(false) TimeZone.setDefault(originalTimeZone) @@ -751,7 +751,7 @@ class HiveWindowFunctionQueryFileSuite private val originalLocale = Locale.getDefault private val testTempDir = Utils.createTempDir() - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) @@ -769,7 +769,7 @@ class HiveWindowFunctionQueryFileSuite // sql("set mapreduce.jobtracker.address=local") } - override def afterAll() { + override def afterAll(): Unit = { try { TestHive.setCacheTables(false) TimeZone.setDefault(originalTimeZone) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d37f0c8573659..f627227aa0380 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -103,14 +103,6 @@ ${hive.group} hive-metastore - - ${hive.group} - hive-contrib - - - ${hive.group}.hcatalog - hive-hcatalog-core -
Optionvaluedefaultquery typemeaning
startingOffsetsByTimestampjson string + """ {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """ + none (the value of startingOffsets will apply)streaming and batchThe start point of timestamp when a query is started, a json string specifying a starting timestamp for + each TopicPartition. The returned offset for each partition is the earliest offset whose timestamp is greater than or + equal to the given timestamp in the corresponding partition. If the matched offset doesn't exist, + the query will fail immediately to prevent unintended read from such partition. (This is a kind of limitation as of now, and will be addressed in near future.)

+

+ Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.

+ For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.

+ Also the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.

+ Note: This option requires Kafka 0.10.1.0 or higher.

+ Note2: startingOffsetsByTimestamp takes precedence over startingOffsets.

+ Note3: For streaming queries, this only applies when a new query is started, and that resuming will + always pick up from where the query left off. Newly discovered partitions during a query will start at + earliest.

startingOffsets "earliest", "latest" (streaming only), or json string @@ -377,6 +398,25 @@ The following configurations are optional: always pick up from where the query left off. Newly discovered partitions during a query will start at earliest.
endingOffsetsByTimestampjson string + """ {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """ + latestbatch queryThe end point when a batch query is ended, a json string specifying an ending timesamp for each TopicPartition. + The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to + the given timestamp in the corresponding partition. If the matched offset doesn't exist, the offset will + be set to latest.

+

+ Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.

+ For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.

+ Also the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.

+ Note: This option requires Kafka 0.10.1.0 or higher.

+ Note2: endingOffsetsByTimestamp takes precedence over endingOffsets. +

endingOffsets latest or json string diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index deaf262c5f572..2a405f36fd5fd 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1505,7 +1505,6 @@ Additional details on supported joins: - Cannot use mapGroupsWithState and flatMapGroupsWithState in Update mode before 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. @@ -1616,6 +1615,8 @@ this configuration judiciously. ### Arbitrary Stateful Operations Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation `mapGroupsWithState` and the more powerful operation `flatMapGroupsWithState`. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html)) and the examples ([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)). +Though Spark cannot check and force it, the state function should be implemented with respect to the semantics of the output mode. For example, in Update mode Spark doesn't expect that the state function will emit rows which are older than current watermark plus allowed late record delay, whereas in Append mode the state function can emit these rows. + ### Unsupported Operations There are a few DataFrame/Dataset operations that are not supported with streaming DataFrames/Datasets. Some of them are as follows. @@ -1647,6 +1648,26 @@ For example, sorting on the input stream is not supported, as it requires keepin track of all the data received in the stream. This is therefore fundamentally hard to execute efficiently. +### Limitation of global watermark + +In Append mode, if a stateful operation emits rows older than current watermark plus allowed late record delay, +they will be "late rows" in downstream stateful operations (as Spark uses global watermark). Note that these rows may be discarded. +This is a limitation of a global watermark, and it could potentially cause a correctness issue. + +Spark will check the logical plan of query and log a warning when Spark detects such a pattern. + +Any of the stateful operation(s) after any of below stateful operations can have this issue: + +* streaming aggregation in Append mode +* stream-stream outer join +* `mapGroupsWithState` and `flatMapGroupsWithState` in Append mode (depending on the implementation of the state function) + +As Spark cannot check the state function of `mapGroupsWithState`/`flatMapGroupsWithState`, Spark assumes that the state function +emits late rows if the operator uses Append mode. + +There's a known workaround: split your streaming query into multiple queries per stateful operator, and ensure +end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional. + ## Starting Streaming Queries Once you have defined the final result DataFrame/Dataset, all that is left is for you to start the streaming computation. To do that, you have to use the `DataStreamWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) diff --git a/examples/pom.xml b/examples/pom.xml index ac148ef4c9c01..a099f1e042e99 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -107,7 +107,7 @@ com.github.scopt scopt_${scala.binary.version} - 3.7.0 + 3.7.1 ${hive.parquet.group} diff --git a/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala index 5d9a9a73f12ec..a0e8d63133b74 100644 --- a/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.SparkSession * 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 only argument, numElem, sets the number elements in the collection to parallelize. * * 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 @@ -36,7 +36,7 @@ import org.apache.spark.sql.SparkSession * accumulator source) are reported to stdout as well. */ object AccumulatorMetricsTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder() diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 3311de12dbd97..d7e79966037cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.SparkSession * Usage: BroadcastTest [partitions] [numElem] [blockSize] */ object BroadcastTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val blockSize = if (args.length > 2) args(2) else "4096" diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala index d12ef642bd2cd..ed56108f4b624 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils * test driver submission in the standalone scheduler. */ object DriverSubmissionTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 1) { println("Usage: DriverSubmissionTest ") System.exit(0) diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index 45c4953a84be2..6e95318a8cbc0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples import org.apache.spark.sql.SparkSession object ExceptionHandlingTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("ExceptionHandlingTest") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 2f2bbb1275438..c07c1afbcb174 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] */ object GroupByTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("GroupBy Test") 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 b327e13533b81..48698678571e3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession object HdfsTest { /** Usage: HdfsTest [file] */ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 1) { System.err.println("Usage: HdfsTest ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 3f9cea35d6503..87c2f6853807a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -93,7 +93,7 @@ object LocalALS { new CholeskyDecomposition(XtX).getSolver.solve(Xty) } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! |Please use org.apache.spark.ml.recommendation.ALS @@ -101,7 +101,7 @@ object LocalALS { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { args match { case Array(m, u, f, iters) => diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index 5512e33e41ac3..5478c585a959e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -39,7 +39,7 @@ object LocalFileLR { DataPoint(new DenseVector(nums.slice(1, D + 1)), nums(0)) } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use org.apache.spark.ml.classification.LogisticRegression @@ -47,7 +47,7 @@ object LocalFileLR { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { showWarning() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index f5162a59522f0..4a73466841f69 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -62,7 +62,7 @@ object LocalKMeans { bestIndex } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! |Please use org.apache.spark.ml.clustering.KMeans @@ -70,7 +70,7 @@ object LocalKMeans { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { showWarning() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index bde8ccd305960..4ca0ecdcfe6e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -46,7 +46,7 @@ object LocalLR { Array.tabulate(N)(generatePoint) } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use org.apache.spark.ml.classification.LogisticRegression @@ -54,7 +54,7 @@ object LocalLR { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { showWarning() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala index a93c15c85cfc1..7660ffd02ed9b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples import scala.math.random object LocalPi { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { var count = 0 for (i <- 1 to 100000) { val x = random * 2 - 1 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 03187aee044e4..e2120eaee6e5a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -41,7 +41,7 @@ object LogQuery { | 0 73.23.2.15 images.com 1358492557 - Whatup""".stripMargin.split('\n').mkString ) - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val sparkConf = new SparkConf().setAppName("Log Query") val sc = new SparkContext(sparkConf) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index e6f33b7adf5d1..4bea5cae775cb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession * Usage: MultiBroadcastTest [partitions] [numElem] */ object MultiBroadcastTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 2332a661f26a0..2bd7c3e954396 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession * Usage: SimpleSkewedGroupByTest [numMappers] [numKVPairs] [valSize] [numReducers] [ratio] */ object SimpleSkewedGroupByTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("SimpleSkewedGroupByTest") diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 4d3c34041bc17..2e7abd62dcdc6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] */ object SkewedGroupByTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("GroupBy Test") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index d3e7b7a967de7..651f0224d4402 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -78,7 +78,7 @@ object SparkALS { new CholeskyDecomposition(XtX).getSolver.solve(Xty) } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! |Please use org.apache.spark.ml.recommendation.ALS @@ -86,7 +86,7 @@ object SparkALS { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { var slices = 0 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 23eaa879114a9..8c09ce614d931 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -49,7 +49,7 @@ object SparkHdfsLR { DataPoint(new DenseVector(x), y) } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use org.apache.spark.ml.classification.LogisticRegression @@ -57,7 +57,7 @@ object SparkHdfsLR { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: SparkHdfsLR ") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index b005cb6971c16..ec9b44ce6e3b7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -49,7 +49,7 @@ object SparkKMeans { bestIndex } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! |Please use org.apache.spark.ml.clustering.KMeans @@ -57,7 +57,7 @@ object SparkKMeans { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 3) { System.err.println("Usage: SparkKMeans ") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 4b1497345af82..deb6668f7ecfc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -51,7 +51,7 @@ object SparkLR { Array.tabulate(N)(generatePoint) } - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use org.apache.spark.ml.classification.LogisticRegression @@ -59,7 +59,7 @@ object SparkLR { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { showWarning() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 9299bad5d3290..3bd475c440d72 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.SparkSession */ object SparkPageRank { - def showWarning() { + def showWarning(): Unit = { System.err.println( """WARN: This is a naive implementation of PageRank and is given as an example! |Please use the PageRank implementation found in org.apache.spark.graphx.lib.PageRank @@ -47,7 +47,7 @@ object SparkPageRank { """.stripMargin) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 1) { System.err.println("Usage: SparkPageRank ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 828d98b5001d7..a8eec6a99cf4b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.SparkSession /** Computes an approximation to pi */ object SparkPi { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("Spark Pi") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkRemoteFileTest.scala b/examples/src/main/scala/org/apache/spark/examples/SparkRemoteFileTest.scala index 64076f2deb706..99a12b9442365 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkRemoteFileTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkRemoteFileTest.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.SparkSession /** Usage: SparkRemoteFileTest [file] */ object SparkRemoteFileTest { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 1) { System.err.println("Usage: SparkRemoteFileTest ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index f5d42141f5dd2..7a6fa9a797ff9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -41,7 +41,7 @@ object SparkTC { edges.toSeq } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("SparkTC") diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index da3ffca1a6f2a..af18c0afbb223 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -23,7 +23,7 @@ package org.apache.spark.examples.graphx * http://snap.stanford.edu/data/soc-LiveJournal1.html. */ object LiveJournalPageRank { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 1) { System.err.println( "Usage: LiveJournalPageRank \n" + diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 57b2edf992208..8bc9c0a86eab6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -47,7 +47,7 @@ object SynthBenchmark { * -degFile the local file to save the degree information (Default: Empty) * -seed seed to use for RNGs (Default: -1, picks seed randomly) */ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val options = args.map { arg => arg.dropWhile(_ == '-').split('=') match { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index 8091838a2301e..354e65c2bae38 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -42,7 +42,7 @@ object ALSExample { } // $example off$ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("ALSExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 5638e66b8792a..1a67a6e755ab4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -25,7 +25,7 @@ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.sql.SparkSession object ChiSqSelectorExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("ChiSqSelectorExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 91d861dd4380a..947ca5f5fb5e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel} import org.apache.spark.sql.SparkSession object CountVectorizerExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("CountVectorizerExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index ee4469faab3a0..4377efd9e95fa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -41,7 +41,7 @@ object DataFrameExample { case class Params(input: String = "data/mllib/sample_libsvm_data.txt") extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("DataFrameExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 19f2d7751bc54..ef38163d7eb0d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -65,7 +65,7 @@ object DecisionTreeExample { checkpointDir: Option[String] = None, checkpointInterval: Int = 10) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("DecisionTreeExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 2dc11b07d88ef..9b5dfed0cb31b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.{Dataset, Row, SparkSession} */ object DeveloperApiExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("DeveloperApiExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala index 8f3ce4b315bd3..ca4235d53e636 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -63,7 +63,7 @@ object GBTExample { checkpointDir: Option[String] = None, checkpointInterval: Int = 10) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("GBTExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala index 2940682c32801..b3642c0b45db6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -25,7 +25,7 @@ import org.apache.spark.ml.feature.{IndexToString, StringIndexer} import org.apache.spark.sql.SparkSession object IndexToStringExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("IndexToStringExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala index 6903a1c298ced..370c6fd7c17fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala @@ -50,7 +50,7 @@ object LinearRegressionExample { tol: Double = 1E-6, fracTest: Double = 0.2) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("LinearRegressionExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala index bd6cc8cff2348..b64ab4792add4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala @@ -55,7 +55,7 @@ object LogisticRegressionExample { tol: Double = 1E-6, fracTest: Double = 0.2) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("LogisticRegressionExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 4ad6c7c3ef202..86e70e8ab0189 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.SparkSession */ object OneVsRestExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName(s"OneVsRestExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index 0fe16fb6dfa9f..55823fe1832e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -23,7 +23,7 @@ import org.apache.spark.ml.feature.QuantileDiscretizer import org.apache.spark.sql.SparkSession object QuantileDiscretizerExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("QuantileDiscretizerExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala index 3c127a46e1f10..6ba14bcd1822f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -64,7 +64,7 @@ object RandomForestExample { checkpointDir: Option[String] = None, checkpointInterval: Int = 10) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("RandomForestExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala index bb4587b82cb37..bf6a4846b6e34 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.feature.SQLTransformer import org.apache.spark.sql.SparkSession object SQLTransformerExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("SQLTransformerExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index ec2df2ef876ba..6121c81cd1f5d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.SparkSession object TfIdfExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("TfIdfExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala index b4179ecc1e56d..05f2ee3288624 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala @@ -82,7 +82,7 @@ object UnaryTransformerExample { object MyTransformer extends DefaultParamsReadable[MyTransformer] // $example off$ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder() .appName("UnaryTransformerExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index 4bcc6ac6a01f5..8ff0e8c6a51c8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.SparkSession object Word2VecExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("Word2Vec example") diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala index a07535bb5a38d..1a7839414b38e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset object AssociationRulesExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("AssociationRulesExample") val sc = new SparkContext(conf) 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 e3cc1d9c83361..6fc3501fc57b5 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 @@ -58,7 +58,7 @@ object BinaryClassification { regType: RegType = L2, regParam: Double = 0.01) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("BinaryClassification") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala index 53d0b8fc208ef..b7f0ba00f913e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala @@ -34,7 +34,7 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} */ object BisectingKMeansExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val sparkConf = new SparkConf().setAppName("mllib.BisectingKMeansExample") val sc = new SparkContext(sparkConf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala index 0b44c339ef139..cf9f7adbf6999 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala @@ -37,7 +37,7 @@ object Correlations { case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala index 681465d2176d4..9082f0b5a8b85 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala @@ -45,7 +45,7 @@ object CosineSimilarity { case class Params(inputFile: String = null, threshold: Double = 0.1) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("CosineSimilarity") { 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 b5d1b02f92524..1029ca04c348f 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 @@ -67,7 +67,7 @@ object DecisionTreeRunner { checkpointDir: Option[String] = None, checkpointInterval: Int = 10) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("DecisionTreeRunner") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala index b228827e5886f..0259df2799174 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -47,7 +47,7 @@ object DenseKMeans { numIterations: Int = 10, initializationMode: InitializationMode = Parallel) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("DenseKMeans") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala index f724ee1030f04..a25ce826ee842 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala @@ -35,7 +35,7 @@ object FPGrowthExample { minSupport: Double = 0.3, numPartition: Int = -1) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("FPGrowthExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala index b1b3a79d87ae1..103d212a80e78 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.Vectors object GaussianMixtureExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("GaussianMixtureExample") val sc = new SparkContext(conf) 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 3f264933cd3cc..12e0c8df274b2 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 @@ -50,7 +50,7 @@ object GradientBoostedTreesRunner { numIterations: Int = 10, fracTest: Double = 0.2) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("GradientBoostedTrees") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala index 9b3c3266ee30a..8435209377553 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala @@ -29,7 +29,7 @@ import org.apache.spark.rdd.RDD object HypothesisTestingExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("HypothesisTestingExample") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala index b0a6f1671a898..17ebd4159b8d7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.Vectors object KMeansExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("KMeansExample") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index cd77ecf990b3b..605ca68e627ec 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -53,7 +53,7 @@ object LDAExample { checkpointDir: Option[String] = None, checkpointInterval: Int = 10) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("LDAExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala index d25962c5500ed..55a45b302b5a3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.Vectors object LatentDirichletAllocationExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("LatentDirichletAllocationExample") val sc = new SparkContext(conf) 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 fd810155d6a88..92c85c9271a5a 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 @@ -48,7 +48,7 @@ object MovieLensALS { numProductBlocks: Int = -1, implicitPrefs: Boolean = false) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("MovieLensALS") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala index f9e47e485e72f..b5c52f9a31224 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala @@ -38,7 +38,7 @@ object MultivariateSummarizer { case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 65603252c4384..eaf1dacd0160a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -62,7 +62,7 @@ object PowerIterationClusteringExample { maxIterations: Int = 15 ) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("PowerIterationClusteringExample") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala index 8b789277774af..1b5d919a047e8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala @@ -25,7 +25,7 @@ import org.apache.spark.mllib.fpm.PrefixSpan object PrefixSpanExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("PrefixSpanExample") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala index 7ccbb5a0640cd..aee12a1b4751f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala @@ -31,7 +31,7 @@ import org.apache.spark.rdd.RDD */ object RandomRDDGeneration { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName(s"RandomRDDGeneration") val sc = new SparkContext(conf) 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 ea13ec05e2fad..2845028dd0814 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 @@ -25,7 +25,7 @@ import org.apache.spark.mllib.recommendation.{ALS, Rating} import org.apache.spark.sql.SparkSession object RankingMetricsExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder .appName("RankingMetricsExample") diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala index ba3deae5d688f..fdde47d60c544 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala @@ -35,7 +35,7 @@ object SampledRDDs { case class Params(input: String = "data/mllib/sample_binary_classification_data.txt") extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("SampledRDDs") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala index 694c3bb18b045..ba16e8f5ff347 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.RDD object SimpleFPGrowth { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("SimpleFPGrowth") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index b76add2f9bc99..b501f4db2efbb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -40,7 +40,7 @@ object SparseNaiveBayes { numFeatures: Int = -1, lambda: Double = 1.0) extends AbstractParams[Params] - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val defaultParams = Params() val parser = new OptionParser[Params]("SparseNaiveBayes") { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala index 7888af79f87f4..5186f599d9628 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala @@ -52,7 +52,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} */ object StreamingKMeansExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 5) { System.err.println( "Usage: StreamingKMeansExample " + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala index a8b144a197229..4c72f444ff9ec 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala @@ -46,7 +46,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} */ object StreamingLogisticRegression { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 4) { System.err.println( diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala index ae4dee24c6474..f60b10a02274b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils */ object StreamingTestExample { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 3) { // scalastyle:off println System.err.println( diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 071d341b81614..6b839f3f4ac1e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -35,7 +35,7 @@ import org.apache.spark.mllib.linalg.distributed.RowMatrix * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). */ object TallSkinnyPCA { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 1) { System.err.println("Usage: TallSkinnyPCA ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 8ae6de16d80e7..8874c2eda3d2e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -35,7 +35,7 @@ import org.apache.spark.mllib.linalg.distributed.RowMatrix * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). */ object TallSkinnySVD { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 1) { System.err.println("Usage: TallSkinnySVD ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index deaa9f252b9b0..4fd482d5b8bf7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.SparkSession case class Record(key: Int, value: String) object RDDRelation { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { // $example on:init_session$ val spark = SparkSession .builder 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 c7b6a50f0ae7c..d4c05e5ad9944 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 @@ -24,7 +24,7 @@ object SQLDataSourceExample { case class Person(name: String, age: Long) - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val spark = SparkSession .builder() .appName("Spark SQL data sources example") 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 678cbc64aff1f..fde281087c267 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 @@ -34,7 +34,7 @@ object SparkSQLExample { case class Person(name: String, age: Long) // $example off:create_ds$ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { // $example on:init_session$ val spark = SparkSession .builder() 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 a832276602b88..3be8a3862f39c 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 @@ -28,7 +28,7 @@ object SparkHiveExample { case class Record(key: Int, value: String) // $example off:spark_hive$ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { // 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. diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala index de477c5ce8161..6dbc70bd141f3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.SparkSession * localhost 9999` */ object StructuredNetworkWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: StructuredNetworkWordCount ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala index b4dad21dd75b0..4ba2c6bc68918 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala @@ -48,7 +48,7 @@ import org.apache.spark.sql.functions._ */ object StructuredNetworkWordCountWindowed { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 3) { System.err.println("Usage: StructuredNetworkWordCountWindowed " + " []") 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 fc3f8fa53c7ae..0f47deaf1021b 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 @@ -37,7 +37,7 @@ import org.apache.spark.streaming.receiver.Receiver * `$ bin/run-example org.apache.spark.examples.streaming.CustomReceiver localhost 9999` */ object CustomReceiver { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: CustomReceiver ") System.exit(1) @@ -64,20 +64,20 @@ object CustomReceiver { class CustomReceiver(host: String, port: Int) extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) { - def onStart() { + def onStart(): Unit = { // Start the thread that receives data over a connection new Thread("Socket Receiver") { - override def run() { receive() } + override def run(): Unit = { receive() } }.start() } - def onStop() { + def onStop(): Unit = { // There is nothing much to do as the thread calling receive() // is designed to stop by itself isStopped() returns false } /** Create a socket connection and receive data until receiver is stopped */ - private def receive() { + private def receive(): Unit = { var socket: Socket = null var userInput: String = null try { 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 3024b59480099..6fdb37194ea7d 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 @@ -37,7 +37,7 @@ import org.apache.spark.streaming.kafka010._ * consumer-group topic1,topic2 */ object DirectKafkaWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 3) { System.err.println(s""" |Usage: DirectKafkaWordCount diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKerberizedKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKerberizedKafkaWordCount.scala index b68a59873a8fe..6a35ce9b2a293 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKerberizedKafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKerberizedKafkaWordCount.scala @@ -76,7 +76,7 @@ import org.apache.spark.streaming.kafka010._ * using SASL_SSL in production. */ object DirectKerberizedKafkaWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 3) { System.err.println(s""" |Usage: DirectKerberizedKafkaWordCount diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 1f282d437dc38..19dc7a3cce0ac 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -33,7 +33,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 1) { System.err.println("Usage: HdfsWordCount ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index 15b57fccb4076..26bb51dde3a1d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -34,7 +34,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} * `$ bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999` */ object NetworkWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: NetworkWordCount ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index 19bacd449787b..09eeaf9fa4496 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -25,7 +25,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} object QueueStream { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { StreamingExamples.setStreamingLogLevels() val sparkConf = new SparkConf().setAppName("QueueStream") diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala index 437ccf0898d7c..a20abd6e9d12e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala @@ -37,7 +37,7 @@ import org.apache.spark.util.IntParam * is the Spark Streaming batch duration in milliseconds. */ object RawNetworkGrep { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 4) { System.err.println("Usage: RawNetworkGrep ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index f018f3a26d2e9..243c22e71275c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -139,7 +139,7 @@ object RecoverableNetworkWordCount { ssc } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 4) { System.err.println(s"Your arguments were ${args.mkString("[", ", ", "]")}") System.err.println( diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 787bbec73b28f..778be7baaeeac 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -38,7 +38,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext, Time} */ object SqlNetworkWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: NetworkWordCount ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 2811e67009fb0..46f01edf7deec 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming._ * org.apache.spark.examples.streaming.StatefulNetworkWordCount localhost 9999` */ object StatefulNetworkWordCount { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: StatefulNetworkWordCount ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala index b00f32fb25243..073f9728c68af 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala @@ -25,7 +25,7 @@ import org.apache.spark.internal.Logging object StreamingExamples extends Logging { /** Set reasonable logging levels for streaming if the user has not configured log4j. */ - def setStreamingLogLevels() { + def setStreamingLogLevels(): Unit = { val log4jInitialized = Logger.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { // We first log something to initialize Spark's default logging, then we override the diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 2108bc63edea2..7234f30e7d267 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -81,7 +81,7 @@ object PageViewGenerator { new PageView(page, status, zipCode, id).toString() } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 2) { System.err.println("Usage: PageViewGenerator ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index b8e7c7e9e9152..b51bfacabf4aa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} */ // scalastyle:on object PageViewStream { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 3) { System.err.println("Usage: PageViewStream ") System.err.println(" must be one of pageCounts, slidingPageCounts," + diff --git a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..94137a691e4aa --- /dev/null +++ b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt @@ -0,0 +1,122 @@ +================================================================================================ +SQL Single Numeric Column Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2995 3081 121 5.3 190.4 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2865 2881 23 5.5 182.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2919 2936 23 5.4 185.6 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 3148 3262 161 5.0 200.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2651 2721 99 5.9 168.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2782 2854 103 5.7 176.9 1.0X + + +================================================================================================ +Int and String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of columns 4531 4583 73 2.3 432.1 1.0X + + +================================================================================================ +Partitioned Table Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column 3084 3105 30 5.1 196.1 1.0X +Partition column 3143 3164 30 5.0 199.8 1.0X +Both columns 3272 3339 94 4.8 208.1 0.9X + + +================================================================================================ +Repeated String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 3249 3318 98 3.2 309.8 1.0X + + +================================================================================================ +String with Nulls Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 5308 5335 38 2.0 506.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 4405 4429 33 2.4 420.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 3256 3309 75 3.2 310.5 1.0X + + +================================================================================================ +Single Column Scan From Wide Columns +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of single column 5230 5290 85 0.2 4987.4 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of single column 10206 10329 174 0.1 9733.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of single column 15333 15365 46 0.1 14622.3 1.0X + + diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt index 8a12d9d15d915..7b008a312c320 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -6,37 +6,37 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2958 3072 161 5.3 188.1 1.0X +Sum 3067 3132 91 5.1 195.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2856 2866 15 5.5 181.6 1.0X +Sum 2927 2929 3 5.4 186.1 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2883 2890 10 5.5 183.3 1.0X +Sum 2928 2990 87 5.4 186.2 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3336 3360 34 4.7 212.1 1.0X +Sum 3374 3447 104 4.7 214.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2728 2760 45 5.8 173.5 1.0X +Sum 2896 2901 7 5.4 184.1 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2826 2833 9 5.6 179.7 1.0X +Sum 3004 3006 3 5.2 191.0 1.0X ================================================================================================ @@ -47,7 +47,7 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 4682 4704 30 2.2 446.6 1.0X +Sum of columns 4814 4830 22 2.2 459.1 1.0X ================================================================================================ @@ -58,9 +58,9 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3206 3265 84 4.9 203.8 1.0X -Partition column 3314 3316 3 4.7 210.7 1.0X -Both columns 3353 3367 20 4.7 213.2 1.0X +Data column 3361 3362 1 4.7 213.7 1.0X +Partition column 2999 3013 20 5.2 190.7 1.1X +Both columns 3613 3615 2 4.4 229.7 0.9X ================================================================================================ @@ -71,7 +71,7 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3364 3367 4 3.1 320.8 1.0X +Sum of string length 3415 3416 1 3.1 325.7 1.0X ================================================================================================ @@ -82,19 +82,19 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5403 5410 10 1.9 515.3 1.0X +Sum of string length 5535 5536 2 1.9 527.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4524 4526 2 2.3 431.5 1.0X +Sum of string length 4567 4575 11 2.3 435.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3121 3129 11 3.4 297.7 1.0X +Sum of string length 3248 3268 29 3.2 309.7 1.0X ================================================================================================ @@ -105,18 +105,18 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 4537 4541 6 0.2 4326.9 1.0X +Sum of single column 5486 5497 15 0.2 5232.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 8768 8776 11 0.1 8361.8 1.0X +Sum of single column 10682 10746 90 0.1 10186.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 13042 13064 31 0.1 12437.6 1.0X +Sum of single column 16177 16177 0 0.1 15427.7 1.0X diff --git a/external/avro/benchmarks/AvroWriteBenchmark-jdk11-results.txt b/external/avro/benchmarks/AvroWriteBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..2cf1835013821 --- /dev/null +++ b/external/avro/benchmarks/AvroWriteBenchmark-jdk11-results.txt @@ -0,0 +1,10 @@ +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 3026 3142 164 5.2 192.4 1.0X +Output Single Double Column 3157 3260 145 5.0 200.7 1.0X +Output Int and String Column 6123 6190 94 2.6 389.3 0.5X +Output Partitions 5197 5733 758 3.0 330.4 0.6X +Output Buckets 7074 7285 298 2.2 449.7 0.4X + diff --git a/external/avro/benchmarks/AvroWriteBenchmark-results.txt b/external/avro/benchmarks/AvroWriteBenchmark-results.txt index fb2a77333eec5..20f6ae9099a4d 100644 --- a/external/avro/benchmarks/AvroWriteBenchmark-results.txt +++ b/external/avro/benchmarks/AvroWriteBenchmark-results.txt @@ -1,10 +1,10 @@ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Avro writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 3080 3137 82 5.1 195.8 1.0X +Output Single Double Column 3595 3595 0 4.4 228.6 0.9X +Output Int and String Column 7491 7504 18 2.1 476.3 0.4X +Output Partitions 5518 5663 205 2.9 350.8 0.6X +Output Buckets 7467 7581 161 2.1 474.7 0.4X 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 cf88981b1efbd..dc60cfe41ca7a 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 @@ -1036,7 +1036,7 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession { (TimestampType, LONG), (DecimalType(4, 2), BYTES) ) - def assertException(f: () => AvroSerializer) { + def assertException(f: () => AvroSerializer): Unit = { val message = intercept[org.apache.spark.sql.avro.IncompatibleSchemaException] { f() }.getMessage diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index a4956ff5ee9cc..aff79b8b8e642 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -106,6 +106,14 @@ test-jar test + + + org.glassfish.jersey.bundles.repackaged + jersey-guava + 2.25.1 + test + mysql mysql-connector-java 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 9cd5c4ec41a52..bba1b5275269b 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 @@ -27,7 +27,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "mysql:5.7.9" + override val imageName = "mysql:5.7.28" override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) @@ -39,6 +39,8 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { } override def dataPreparation(conn: Connection): Unit = { + // Since MySQL 5.7.14+, we need to disable strict mode + conn.prepareStatement("SET GLOBAL sql_mode = ''").executeUpdate() conn.prepareStatement("CREATE DATABASE foo").executeUpdate() conn.prepareStatement("CREATE TABLE tbl (x INTEGER, y TEXT(8))").executeUpdate() conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate() 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 89da9a1de6f74..599f00def0750 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 @@ -29,7 +29,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "postgres:11.4" + override val imageName = "postgres:12.0-alpine" override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 0735f0a7b937f..693820da6af6b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -46,6 +46,13 @@ ${project.version} provided + + org.apache.spark + spark-token-provider-kafka-0-10_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-core_${scala.binary.version} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala index 868edb5dcdc0c..6dd5af2389a81 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala @@ -68,7 +68,7 @@ private object JsonUtils { partOffsets.map { case (part, offset) => new TopicPartition(topic, part) -> offset } - }.toMap + } } catch { case NonFatal(x) => throw new IllegalArgumentException( @@ -76,12 +76,27 @@ private object JsonUtils { } } + def partitionTimestamps(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partTimestamps) => + partTimestamps.map { case (part, timestamp) => + new TopicPartition(topic, part) -> timestamp + } + } + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA": {"0": 123456789, "1": 123456789}, + |"topicB": {"0": 123456789, "1": 123456789}}, got $str""".stripMargin) + } + } + /** * Write per-TopicPartition offsets as json string */ def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { val result = new HashMap[String, HashMap[Int, Long]]() - implicit val ordering = new Ordering[TopicPartition] { + implicit val order = new Ordering[TopicPartition] { override def compare(x: TopicPartition, y: TopicPartition): Int = { Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition)) } @@ -95,4 +110,9 @@ private object JsonUtils { } Serialization.write(result) } + + def partitionTimestamps(topicTimestamps: Map[TopicPartition, Long]): String = { + // For now it's same as partitionOffsets + partitionOffsets(topicTimestamps) + } } 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 index 667c383681917..3006770f306c0 100644 --- 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 @@ -59,8 +59,8 @@ private[kafka010] class KafkaBatch( // Leverage the KafkaReader to obtain the relevant partition offsets val (fromPartitionOffsets, untilPartitionOffsets) = { try { - (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets), - kafkaOffsetReader.fetchPartitionOffsets(endingOffsets)) + (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, isStartingOffsets = true), + kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, isStartingOffsets = false)) } finally { kafkaOffsetReader.close() } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index 9e7b7d6db2038..0603ae39ba622 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -69,6 +69,8 @@ class KafkaContinuousStream( case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) + case SpecificTimestampRangeLimit(p) => offsetReader.fetchSpecificTimestampBasedOffsets(p, + failsOnNoMatchingOffset = true) } logInfo(s"Initial offsets: $offsets") offsets 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 87036beb9a252..ca82c908f441b 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 @@ -23,12 +23,13 @@ import java.util.concurrent.TimeoutException import scala.collection.JavaConverters._ +import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException} import org.apache.kafka.common.TopicPartition import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.kafka010.KafkaConfigUpdater +import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaTokenClusterConf, KafkaTokenUtil} import org.apache.spark.sql.kafka010.KafkaDataConsumer.{AvailableOffsetRange, UNKNOWN_OFFSET} import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.util.{ShutdownHookManager, UninterruptibleThread} @@ -46,6 +47,13 @@ private[kafka010] class InternalKafkaConsumer( val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + private[kafka010] val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig( + SparkEnv.get.conf, kafkaParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + .asInstanceOf[String]) + + // Kafka consumer is not able to give back the params instantiated with so we need to store it. + // It must be updated whenever a new consumer is created. + private[kafka010] var kafkaParamsWithSecurity: ju.Map[String, Object] = _ private val consumer = createConsumer() /** @@ -106,10 +114,10 @@ private[kafka010] class InternalKafkaConsumer( /** Create a KafkaConsumer to fetch records for `topicPartition` */ private def createConsumer(): KafkaConsumer[Array[Byte], Array[Byte]] = { - val updatedKafkaParams = KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) - .setAuthenticationConfigIfNeeded() + kafkaParamsWithSecurity = KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded(clusterConfig) .build() - val c = new KafkaConsumer[Array[Byte], Array[Byte]](updatedKafkaParams) + val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParamsWithSecurity) val tps = new ju.ArrayList[TopicPartition]() tps.add(topicPartition) c.assign(tps) @@ -516,13 +524,25 @@ private[kafka010] class KafkaDataConsumer( fetchedData.withNewPoll(records.listIterator, offsetAfterPoll) } - private def getOrRetrieveConsumer(): InternalKafkaConsumer = _consumer match { - case None => - _consumer = Option(consumerPool.borrowObject(cacheKey, kafkaParams)) - require(_consumer.isDefined, "borrowing consumer from pool must always succeed.") - _consumer.get + private[kafka010] def getOrRetrieveConsumer(): InternalKafkaConsumer = { + if (!_consumer.isDefined) { + retrieveConsumer() + } + require(_consumer.isDefined, "Consumer must be defined") + if (!KafkaTokenUtil.isConnectorUsingCurrentToken(_consumer.get.kafkaParamsWithSecurity, + _consumer.get.clusterConfig)) { + logDebug("Cached consumer uses an old delegation token, invalidating.") + releaseConsumer() + consumerPool.invalidateKey(cacheKey) + fetchedDataPool.invalidate(cacheKey) + retrieveConsumer() + } + _consumer.get + } - case Some(consumer) => consumer + private def retrieveConsumer(): Unit = { + _consumer = Option(consumerPool.borrowObject(cacheKey, kafkaParams)) + require(_consumer.isDefined, "borrowing consumer from pool must always succeed.") } private def getOrRetrieveFetchedData(offset: Long): FetchedData = _fetchedData match { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 6ea6efe5d1b74..01f6ba4445162 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -192,6 +192,8 @@ private[kafka010] class KafkaMicroBatchStream( KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss) + case SpecificTimestampRangeLimit(p) => + kafkaOffsetReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala index 80a026f4f5d73..d64b5d4f7e9e8 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala @@ -42,6 +42,13 @@ private[kafka010] case object LatestOffsetRangeLimit extends KafkaOffsetRangeLim private[kafka010] case class SpecificOffsetRangeLimit( partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit +/** + * Represents the desire to bind to earliest offset which timestamp for the offset is equal or + * greater than specific timestamp. + */ +private[kafka010] case class SpecificTimestampRangeLimit( + topicTimestamps: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit + private[kafka010] object KafkaOffsetRangeLimit { /** * Used to denote offset range limits that are resolved via Kafka 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 20f2ce11d4afb..0179f4dd822f1 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 @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import scala.util.control.NonFatal -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetAndTimestamp} import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging @@ -126,12 +126,14 @@ private[kafka010] class KafkaOffsetReader( * Fetch the partition offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]] and [[KafkaOffsetRangeLimit]]. */ - def fetchPartitionOffsets(offsetRangeLimit: KafkaOffsetRangeLimit): Map[TopicPartition, Long] = { + def fetchPartitionOffsets( + offsetRangeLimit: KafkaOffsetRangeLimit, + isStartingOffsets: Boolean): 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" + + "Use -1 for latest, -2 for earliest.\n" + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions}") logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") partitionOffsets @@ -147,6 +149,9 @@ private[kafka010] class KafkaOffsetReader( }.toMap case SpecificOffsetRangeLimit(partitionOffsets) => validateTopicPartitions(partitions, partitionOffsets) + case SpecificTimestampRangeLimit(partitionTimestamps) => + fetchSpecificTimestampBasedOffsets(partitionTimestamps, + failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets } } @@ -161,23 +166,83 @@ private[kafka010] class KafkaOffsetReader( def fetchSpecificOffsets( partitionOffsets: Map[TopicPartition, Long], reportDataLoss: String => Unit): KafkaSourceOffset = { - val fetched = runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() + val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions => + assert(partitions.asScala == 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.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + } - // 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(_ => {}) + val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => + partitionOffsets + } + + val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched => + partitionOffsets.foreach { + case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && + off != KafkaOffsetRangeLimit.EARLIEST => + if (fetched(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${fetched(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + } + + fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets, + fnAssertFetchedOffsets) + } + + def fetchSpecificTimestampBasedOffsets( + partitionTimestamps: Map[TopicPartition, Long], + failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = { + val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions => + assert(partitions.asScala == partitionTimestamps.keySet, + "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " + + s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps") + } - consumer.pause(partitions) - assert(partitions.asScala == 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.asScala}") - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => { + val converted = partitionTimestamps.map { case (tp, timestamp) => + tp -> java.lang.Long.valueOf(timestamp) + }.asJava + + val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] = + consumer.offsetsForTimes(converted) + + offsetForTime.asScala.map { case (tp, offsetAndTimestamp) => + if (failsOnNoMatchingOffset) { + assert(offsetAndTimestamp != null, "No offset matched from request of " + + s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.") + } + + if (offsetAndTimestamp == null) { + tp -> KafkaOffsetRangeLimit.LATEST + } else { + tp -> offsetAndTimestamp.offset() + } + }.toMap + } + } + + val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => } + + fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets, + fnAssertFetchedOffsets) + } + + private def fetchSpecificOffsets0( + fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit, + fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long], + fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit): KafkaSourceOffset = { + val fetched = partitionsAssignedToConsumer { + partitions => { + fnAssertParametersWithPartitions(partitions) + + val partitionOffsets = fnRetrievePartitionOffsets(partitions) partitionOffsets.foreach { case (tp, KafkaOffsetRangeLimit.LATEST) => @@ -186,22 +251,15 @@ private[kafka010] class KafkaOffsetReader( consumer.seekToBeginning(ju.Arrays.asList(tp)) case (tp, off) => consumer.seek(tp, off) } + partitionOffsets.map { case (tp, _) => tp -> consumer.position(tp) } } } - partitionOffsets.foreach { - case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && - off != KafkaOffsetRangeLimit.EARLIEST => - if (fetched(tp) != off) { - reportDataLoss( - s"startingOffsets for $tp was $off but consumer reset to ${fetched(tp)}") - } - case _ => - // no real way to check that beginning or end is reasonable - } + fnAssertFetchedOffsets(fetched) + KafkaSourceOffset(fetched) } @@ -209,20 +267,15 @@ private[kafka010] class KafkaOffsetReader( * Fetch the earliest offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]]. */ - def fetchEarliestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") + def fetchEarliestOffsets(): Map[TopicPartition, Long] = partitionsAssignedToConsumer( + partitions => { + logDebug("Seeking to the beginning") consumer.seekToBeginning(partitions) val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap logDebug(s"Got earliest offsets for partition : $partitionOffsets") partitionOffsets - } - } + }, fetchingEarliestOffset = true) /** * Fetch the latest offsets for the topic partitions that are indicated @@ -239,19 +292,9 @@ private[kafka010] class KafkaOffsetReader( * distinguish this with KAFKA-7703, so we just return whatever we get from Kafka after retrying. */ 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.") + knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = + partitionsAssignedToConsumer { partitions => { + logDebug("Seeking to the end.") if (knownOffsets.isEmpty) { consumer.seekToEnd(partitions) @@ -315,25 +358,40 @@ private[kafka010] class KafkaOffsetReader( if (newPartitions.isEmpty) { Map.empty[TopicPartition, Long] } else { - runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"\tPartitions assigned to consumer: $partitions") - - // Get the earliest offset of each partition - consumer.seekToBeginning(partitions) - val partitionOffsets = newPartitions.filter { p => - // When deleting topics happen at the same time, some partitions may not be in - // `partitions`. So we need to ignore them - partitions.contains(p) - }.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") - partitionOffsets - } + partitionsAssignedToConsumer(partitions => { + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in + // `partitions`. So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") + partitionOffsets + }, fetchingEarliestOffset = true) + } + } + + private def partitionsAssignedToConsumer( + body: ju.Set[TopicPartition] => Map[TopicPartition, Long], + fetchingEarliestOffset: Boolean = false) + : Map[TopicPartition, Long] = runUninterruptibly { + + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + + if (!fetchingEarliestOffset) { + // 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.") + body(partitions) } } 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 886f6b0fe0a5e..61479c992039b 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 @@ -68,8 +68,8 @@ private[kafka010] class KafkaRelation( // Leverage the KafkaReader to obtain the relevant partition offsets val (fromPartitionOffsets, untilPartitionOffsets) = { try { - (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets), - kafkaOffsetReader.fetchPartitionOffsets(endingOffsets)) + (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, isStartingOffsets = true), + kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, isStartingOffsets = false)) } finally { kafkaOffsetReader.close() } 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 29944dc3fbf1e..e1392b6215d3a 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 @@ -105,6 +105,8 @@ private[kafka010] class KafkaSource( case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss) + case SpecificTimestampRangeLimit(p) => + kafkaReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") 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 a7f8db35d7cf9..c15f08d78741d 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 @@ -89,7 +89,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveParameters, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveParameters), @@ -126,11 +127,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + caseInsensitiveParameters, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) assert(startingRelationOffsets != LatestOffsetRangeLimit) val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParameters, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveParameters, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) assert(endingRelationOffsets != EarliestOffsetRangeLimit) val includeHeaders = caseInsensitiveParameters.getOrElse(INCLUDE_HEADERS, "false").toBoolean @@ -321,13 +324,17 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // Stream specific options params.get(ENDING_OFFSETS_OPTION_KEY).map(_ => throw new IllegalArgumentException("ending offset not valid in streaming queries")) + params.get(ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY).map(_ => + throw new IllegalArgumentException("ending timestamp not valid in streaming queries")) + validateGeneralOptions(params) } private def validateBatchOptions(params: CaseInsensitiveMap[String]) = { // Batch specific options KafkaSourceProvider.getKafkaOffsetRangeLimit( - params, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match { + params, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY, + EarliestOffsetRangeLimit) match { case EarliestOffsetRangeLimit => // good to go case LatestOffsetRangeLimit => throw new IllegalArgumentException("starting offset can't be latest " + @@ -339,10 +346,12 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister "be latest for batch queries on Kafka") case _ => // ignore } + case _: SpecificTimestampRangeLimit => // good to go } KafkaSourceProvider.getKafkaOffsetRangeLimit( - params, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match { + params, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, ENDING_OFFSETS_OPTION_KEY, + LatestOffsetRangeLimit) match { case EarliestOffsetRangeLimit => throw new IllegalArgumentException("ending offset can't be earliest " + "for batch queries on Kafka") @@ -354,6 +363,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister "earliest for batch queries on Kafka") case _ => // ignore } + case _: SpecificTimestampRangeLimit => // good to go } validateGeneralOptions(params) @@ -420,10 +430,12 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) new KafkaBatch( strategy(caseInsensitiveOptions), @@ -446,7 +458,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveOptions), @@ -474,7 +487,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveOptions), @@ -500,6 +514,8 @@ private[kafka010] object KafkaSourceProvider extends Logging { 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[kafka010] val STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY = "startingoffsetsbytimestamp" + private[kafka010] val ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY = "endingoffsetsbytimestamp" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" private[kafka010] val MIN_PARTITIONS_OPTION_KEY = "minpartitions" private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxoffsetspertrigger" @@ -543,15 +559,20 @@ private[kafka010] object KafkaSourceProvider extends Logging { def getKafkaOffsetRangeLimit( params: CaseInsensitiveMap[String], + offsetByTimestampOptionKey: String, offsetOptionKey: String, defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = { - params.get(offsetOptionKey).map(_.trim) match { - case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" => - LatestOffsetRangeLimit - case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" => - EarliestOffsetRangeLimit - case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) - case None => defaultOffsets + params.get(offsetByTimestampOptionKey).map(_.trim) match { + case Some(json) => SpecificTimestampRangeLimit(JsonUtils.partitionTimestamps(json)) + case None => + params.get(offsetOptionKey).map(_.trim) match { + case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" => + LatestOffsetRangeLimit + case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" => + EarliestOffsetRangeLimit + case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) + case None => defaultOffsets + } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala index 35c1379de160b..7425a74315e1a 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala @@ -46,7 +46,7 @@ class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTest val producer2 = CachedKafkaProducer.getOrCreate(kafkaParams) assert(producer == producer2) - val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]]('getAsMap) + val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]](Symbol("getAsMap")) val map = CachedKafkaProducer.invokePrivate(cacheMap()) assert(map.size == 1) } @@ -63,7 +63,7 @@ class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTest // With updated conf, a new producer instance should be created. assert(producer != producer2) - val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]]('getAsMap) + val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]](Symbol("getAsMap")) val map = CachedKafkaProducer.invokePrivate(cacheMap()) assert(map.size == 2) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedDataPoolSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedDataPoolSuite.scala index cbe54614ef127..5449f5d733c50 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedDataPoolSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/FetchedDataPoolSuite.scala @@ -41,7 +41,7 @@ class FetchedDataPoolSuite extends SharedSparkSession with PrivateMethodTester { // Helper private method accessors for FetchedDataPool private type PoolCacheType = mutable.Map[CacheKey, CachedFetchedDataList] - private val _cache = PrivateMethod[PoolCacheType]('cache) + private val _cache = PrivateMethod[PoolCacheType](Symbol("cache")) def getCache(pool: FetchedDataPool): PoolCacheType = { pool.invokePrivate(_cache()) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala index d97f627fbac08..d22955180d050 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.kafka010 +import java.{util => ju} import java.nio.charset.StandardCharsets import java.util.concurrent.{Executors, TimeUnit} @@ -30,10 +31,14 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.scalatest.PrivateMethodTester import org.apache.spark.{TaskContext, TaskContextImpl} +import org.apache.spark.kafka010.KafkaDelegationTokenTest import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey import org.apache.spark.sql.test.SharedSparkSession -class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester { +class KafkaDataConsumerSuite + extends SharedSparkSession + with PrivateMethodTester + with KafkaDelegationTokenTest { protected var testUtils: KafkaTestUtils = _ private val topic = "topic" + Random.nextInt() @@ -66,13 +71,16 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester private var consumerPool: InternalKafkaConsumerPool = _ override def beforeEach(): Unit = { + super.beforeEach() + fetchedDataPool = { - val fetchedDataPoolMethod = PrivateMethod[FetchedDataPool]('fetchedDataPool) + val fetchedDataPoolMethod = PrivateMethod[FetchedDataPool](Symbol("fetchedDataPool")) KafkaDataConsumer.invokePrivate(fetchedDataPoolMethod()) } consumerPool = { - val internalKafkaConsumerPoolMethod = PrivateMethod[InternalKafkaConsumerPool]('consumerPool) + val internalKafkaConsumerPoolMethod = + PrivateMethod[InternalKafkaConsumerPool](Symbol("consumerPool")) KafkaDataConsumer.invokePrivate(internalKafkaConsumerPoolMethod()) } @@ -82,7 +90,7 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester test("SPARK-19886: Report error cause correctly in reportDataLoss") { val cause = new Exception("D'oh!") - val reportDataLoss = PrivateMethod[Unit]('reportDataLoss0) + val reportDataLoss = PrivateMethod[Unit](Symbol("reportDataLoss0")) val e = intercept[IllegalStateException] { KafkaDataConsumer.invokePrivate(reportDataLoss(true, "message", cause)) } @@ -96,47 +104,86 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester val context1 = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null) TaskContext.setTaskContext(context1) - val consumer1 = KafkaDataConsumer.acquire(topicPartition, kafkaParams) - - // any method call which requires consumer is necessary - consumer1.getAvailableOffsetRange() - - val consumer1Underlying = consumer1._consumer - assert(consumer1Underlying.isDefined) - - consumer1.release() - - assert(consumerPool.size(key) === 1) - // check whether acquired object is available in pool - val pooledObj = consumerPool.borrowObject(key, kafkaParams) - assert(consumer1Underlying.get.eq(pooledObj)) - consumerPool.returnObject(pooledObj) + val consumer1Underlying = initSingleConsumer(kafkaParams, key) val context2 = new TaskContextImpl(0, 0, 0, 0, 1, null, null, null) TaskContext.setTaskContext(context2) - val consumer2 = KafkaDataConsumer.acquire(topicPartition, kafkaParams) - - // any method call which requires consumer is necessary - consumer2.getAvailableOffsetRange() + val consumer2Underlying = initSingleConsumer(kafkaParams, key) - val consumer2Underlying = consumer2._consumer - assert(consumer2Underlying.isDefined) // here we expect different consumer as pool will invalidate for task reattempt - assert(consumer2Underlying.get.ne(consumer1Underlying.get)) + assert(consumer2Underlying.ne(consumer1Underlying)) + } finally { + TaskContext.unset() + } + } - consumer2.release() + test("same KafkaDataConsumer instance in case of same token") { + try { + val kafkaParams = getKafkaParams() + val key = new CacheKey(groupId, topicPartition) + + val context = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null) + TaskContext.setTaskContext(context) + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers + ) + ) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) + val consumer1Underlying = initSingleConsumer(kafkaParams, key) + val consumer2Underlying = initSingleConsumer(kafkaParams, key) + + assert(consumer2Underlying.eq(consumer1Underlying)) + } finally { + TaskContext.unset() + } + } - // The first consumer should be removed from cache, but the consumer after invalidate - // should be cached. - assert(consumerPool.size(key) === 1) - val pooledObj2 = consumerPool.borrowObject(key, kafkaParams) - assert(consumer2Underlying.get.eq(pooledObj2)) - consumerPool.returnObject(pooledObj2) + test("new KafkaDataConsumer instance in case of token renewal") { + try { + val kafkaParams = getKafkaParams() + val key = new CacheKey(groupId, topicPartition) + + val context = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null) + TaskContext.setTaskContext(context) + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers + ) + ) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) + val consumer1Underlying = initSingleConsumer(kafkaParams, key) + addTokenToUGI(tokenService1, tokenId2, tokenPassword2) + val consumer2Underlying = initSingleConsumer(kafkaParams, key) + + assert(consumer2Underlying.ne(consumer1Underlying)) } finally { TaskContext.unset() } } + private def initSingleConsumer( + kafkaParams: ju.Map[String, Object], + key: CacheKey): InternalKafkaConsumer = { + val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams) + + // any method call which requires consumer is necessary + consumer.getOrRetrieveConsumer() + + val consumerUnderlying = consumer._consumer + assert(consumerUnderlying.isDefined) + + consumer.release() + + assert(consumerPool.size(key) === 1) + // check whether acquired object is available in pool + val pooledObj = consumerPool.borrowObject(key, kafkaParams) + assert(consumerUnderlying.get.eq(pooledObj)) + consumerPool.returnObject(pooledObj) + + consumerUnderlying.get + } + test("SPARK-23623: concurrent use of KafkaDataConsumer") { val data: immutable.IndexedSeq[(String, Seq[(String, Array[Byte])])] = prepareTestTopicHavingTestMessages(topic) @@ -323,7 +370,10 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester private def prepareTestTopicHavingTestMessages(topic: String) = { val data = (1 to 1000).map(i => (i.toString, Seq[(String, Array[Byte])]())) testUtils.createTopic(topic, 1) - testUtils.sendMessages(topic, data.toArray, None) + val messages = data.map { case (value, hdrs) => + new RecordBuilder(topic, value).headers(hdrs).build() + } + testUtils.sendMessages(messages) data } 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 609cf3ce4bd75..3ee59e57a6edf 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 @@ -679,7 +679,8 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { }) } - private def testGroupId(groupIdKey: String, validateGroupId: (String, Iterable[String]) => Unit) { + private def testGroupId(groupIdKey: String, + validateGroupId: (String, Iterable[String]) => Unit): Unit = { // Tests code path KafkaSourceProvider.{sourceSchema(.), createSource(.)} // as well as KafkaOffsetReader.createConsumer(.) val topic = newTopic() @@ -1178,9 +1179,10 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) require(testUtils.getLatestOffsets(Set(topic)).size === 5) + val headers = Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8))) (31 to 35).map { num => - (num - 31, (num.toString, Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8))))) - }.foreach { rec => testUtils.sendMessage(topic, rec._2, Some(rec._1)) } + new RecordBuilder(topic, num.toString).partition(num - 31).headers(headers).build() + }.foreach { rec => testUtils.sendMessage(rec) } val kafka = spark .readStream @@ -1277,6 +1279,16 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { "failOnDataLoss" -> failOnDataLoss.toString) } + test(s"assign from specific timestamps (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromSpecificTimestamps( + topic, + failOnDataLoss = failOnDataLoss, + addPartitions = false, + "assign" -> assignString(topic, 0 to 4), + "failOnDataLoss" -> failOnDataLoss.toString) + } + test(s"subscribing topic by name from latest offsets (failOnDataLoss: $failOnDataLoss)") { val topic = newTopic() testFromLatestOffsets( @@ -1300,6 +1312,12 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { testFromSpecificOffsets(topic, failOnDataLoss = failOnDataLoss, "subscribe" -> topic) } + test(s"subscribing topic by name from specific timestamps (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromSpecificTimestamps(topic, failOnDataLoss = failOnDataLoss, addPartitions = true, + "subscribe" -> topic) + } + test(s"subscribing topic by pattern from latest offsets (failOnDataLoss: $failOnDataLoss)") { val topicPrefix = newTopic() val topic = topicPrefix + "-suffix" @@ -1328,6 +1346,17 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { failOnDataLoss = failOnDataLoss, "subscribePattern" -> s"$topicPrefix-.*") } + + test(s"subscribing topic by pattern from specific timestamps " + + s"(failOnDataLoss: $failOnDataLoss)") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromSpecificTimestamps( + topic, + failOnDataLoss = failOnDataLoss, + addPartitions = true, + "subscribePattern" -> s"$topicPrefix-.*") + } } test("bad source options") { @@ -1347,6 +1376,9 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { // Specifying an ending offset testBadOptions("endingOffsets" -> "latest")("Ending offset not valid in streaming queries") + testBadOptions("subscribe" -> "t", "endingOffsetsByTimestamp" -> "{\"t\": {\"0\": 1000}}")( + "Ending timestamp not valid in streaming queries") + // No strategy specified testBadOptions()("options must be specified", "subscribe", "subscribePattern") @@ -1395,7 +1427,8 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { (STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""", SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) { val offset = getKafkaOffsetRangeLimit( - CaseInsensitiveMap[String](Map(optionKey -> optionValue)), optionKey, answer) + CaseInsensitiveMap[String](Map(optionKey -> optionValue)), "dummy", optionKey, + answer) assert(offset === answer) } @@ -1403,7 +1436,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { (STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit), (ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) { val offset = getKafkaOffsetRangeLimit( - CaseInsensitiveMap[String](Map.empty), optionKey, answer) + CaseInsensitiveMap[String](Map.empty), "dummy", optionKey, answer) assert(offset === answer) } } @@ -1468,12 +1501,89 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { ) } + private def testFromSpecificTimestamps( + topic: String, + failOnDataLoss: Boolean, + addPartitions: Boolean, + options: (String, String)*): Unit = { + def sendMessages(topic: String, msgs: Seq[String], part: Int, ts: Long): Unit = { + val records = msgs.map { msg => + new RecordBuilder(topic, msg).partition(part).timestamp(ts).build() + } + testUtils.sendMessages(records) + } + + testUtils.createTopic(topic, partitions = 5) + + val firstTimestamp = System.currentTimeMillis() - 5000 + sendMessages(topic, Array(-20).map(_.toString), 0, firstTimestamp) + sendMessages(topic, Array(-10).map(_.toString), 1, firstTimestamp) + sendMessages(topic, Array(0, 1).map(_.toString), 2, firstTimestamp) + sendMessages(topic, Array(10, 11).map(_.toString), 3, firstTimestamp) + sendMessages(topic, Array(20, 21, 22).map(_.toString), 4, firstTimestamp) + + val secondTimestamp = firstTimestamp + 1000 + sendMessages(topic, Array(-21, -22).map(_.toString), 0, secondTimestamp) + sendMessages(topic, Array(-11, -12).map(_.toString), 1, secondTimestamp) + sendMessages(topic, Array(2).map(_.toString), 2, secondTimestamp) + sendMessages(topic, Array(12).map(_.toString), 3, secondTimestamp) + // no data after second timestamp for partition 4 + + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + // we intentionally starts from second timestamp, + // except for partition 4 - it starts from first timestamp + val startPartitionTimestamps: Map[TopicPartition, Long] = Map( + (0 to 3).map(new TopicPartition(topic, _) -> secondTimestamp): _* + ) ++ Map(new TopicPartition(topic, 4) -> firstTimestamp) + val startingTimestamps = JsonUtils.partitionTimestamps(startPartitionTimestamps) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffsetsByTimestamp", startingTimestamps) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("failOnDataLoss", failOnDataLoss.toString) + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.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, + Execute { q => + val partitions = (0 to 4).map(new TopicPartition(topic, _)) + // wait to reach the last offset in every partition + q.awaitOffset( + 0, KafkaSourceOffset(partitions.map(tp => tp -> 3L).toMap), streamingTimeout.toMillis) + }, + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22), + StopStream, + StartStream(), + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22), // Should get the data back on recovery + StopStream, + AddKafkaData(Set(topic), 30, 31, 32), // Add data when stream is stopped + StartStream(), + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22, 30, 31, 32), // Should get the added data + AssertOnQuery("Add partitions") { query: StreamExecution => + if (addPartitions) setTopicPartitions(topic, 10, query) + true + }, + AddKafkaData(Set(topic), 40, 41, 42, 43, 44)(ensureDataInMultiplePartition = true), + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22, 30, 31, 32, 40, 41, 42, 43, 44), + StopStream + ) + } + test("Kafka column types") { val now = System.currentTimeMillis() val topic = newTopic() testUtils.createTopic(newTopic(), partitions = 1) testUtils.sendMessage( - topic, ("1", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), None + new RecordBuilder(topic, "1") + .headers(Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))).build() ) val kafka = spark 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 3c88609bcb45d..063e2e2bc8b77 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 @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.util.Locale import java.util.concurrent.atomic.AtomicInteger +import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.util.Random @@ -28,11 +29,11 @@ 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.SparkException +import org.apache.spark.sql.{DataFrameReader, 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.SharedSparkSession import org.apache.spark.util.Utils @@ -159,13 +160,15 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession val topic = newTopic() testUtils.createTopic(topic, partitions = 3) testUtils.sendMessage( - topic, ("1", Seq()), Some(0) + new RecordBuilder(topic, "1").headers(Seq()).partition(0).build() ) testUtils.sendMessage( - topic, ("2", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), Some(1) + new RecordBuilder(topic, "2").headers( + Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))).partition(1).build() ) testUtils.sendMessage( - topic, ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8)))), Some(2) + new RecordBuilder(topic, "3").headers( + Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8)))).partition(2).build() ) // Implicit offset values, should default to earliest and latest @@ -176,6 +179,191 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8))))).toDF) } + test("timestamp provided for starting and ending") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // timestamp both presented: starting "first" ending "finalized" + verifyTimestampRelatedQueryResult({ df => + val startPartitionTimestamps: Map[TopicPartition, Long] = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(1)): _*) + val startingTimestamps = JsonUtils.partitionTimestamps(startPartitionTimestamps) + + val endPartitionTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(2)): _*) + val endingTimestamps = JsonUtils.partitionTimestamps(endPartitionTimestamps) + + df.option("startingOffsetsByTimestamp", startingTimestamps) + .option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, 10 to 19) + } + + test("timestamp provided for starting, offset provided for ending") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // starting only presented as "first", and ending presented as endingOffsets + verifyTimestampRelatedQueryResult({ df => + val startTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps.head): _*) + val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps) + + val endPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -1L, // -1 => latest + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 1L // explicit offset - take only first one + ) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + + // so we here expect full of records from partition 0 and 1, and only the first record + // from partition 2 which is "2" + + df.option("startingOffsetsByTimestamp", startingTimestamps) + .option("endingOffsets", endingOffsets) + }, topic, (0 to 29).filterNot(_ % 3 == 2) ++ Seq(2)) + } + + test("timestamp provided for ending, offset provided for starting") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // ending only presented as "third", and starting presented as startingOffsets + verifyTimestampRelatedQueryResult({ df => + val startPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, // -2 => earliest + new TopicPartition(topic, 1) -> -2L, + new TopicPartition(topic, 2) -> 0L // explicit earliest + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + + val endTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(2)): _*) + val endingTimestamps = JsonUtils.partitionTimestamps(endTopicTimestamps) + + df.option("startingOffsets", startingOffsets) + .option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, 0 to 19) + } + + test("timestamp provided for starting, ending not provided") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // starting only presented as "second", and ending not presented + verifyTimestampRelatedQueryResult({ df => + val startTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(1)): _*) + val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps) + + df.option("startingOffsetsByTimestamp", startingTimestamps) + }, topic, 10 to 29) + } + + test("timestamp provided for ending, starting not provided") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // ending only presented as "third", and starting not presented + verifyTimestampRelatedQueryResult({ df => + val endTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(2)): _*) + val endingTimestamps = JsonUtils.partitionTimestamps(endTopicTimestamps) + + df.option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, 0 to 19) + } + + test("no matched offset for timestamp - startingOffsets") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + val e = intercept[SparkException] { + verifyTimestampRelatedQueryResult({ df => + // partition 2 will make query fail + val startTopicTimestamps = Map( + (0 to 1).map(new TopicPartition(topic, _) -> timestamps(1)): _*) ++ + Map(new TopicPartition(topic, 2) -> Long.MaxValue) + + val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps) + + df.option("startingOffsetsByTimestamp", startingTimestamps) + }, topic, Seq.empty) + } + + @tailrec + def assertionErrorInExceptionChain(e: Throwable): Boolean = { + if (e.isInstanceOf[AssertionError]) { + true + } else if (e.getCause == null) { + false + } else { + assertionErrorInExceptionChain(e.getCause) + } + } + + assert(assertionErrorInExceptionChain(e), + "Cannot find expected AssertionError in chained exceptions") + } + + test("no matched offset for timestamp - endingOffsets") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // the query will run fine, since we allow no matching offset for timestamp + // if it's endingOffsets + // for partition 0 and 1, it only takes records between first and second timestamp + // for partition 2, it will take all records + verifyTimestampRelatedQueryResult({ df => + val endTopicTimestamps = Map( + (0 to 1).map(new TopicPartition(topic, _) -> timestamps(1)): _*) ++ + Map(new TopicPartition(topic, 2) -> Long.MaxValue) + + val endingTimestamps = JsonUtils.partitionTimestamps(endTopicTimestamps) + + df.option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, (0 to 9) ++ (10 to 29).filter(_ % 3 == 2)) + } + + private def prepareTimestampRelatedUnitTest: (String, Seq[Long]) = { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + + def sendMessages(topic: String, msgs: Array[String], part: Int, ts: Long): Unit = { + val records = msgs.map { msg => + new RecordBuilder(topic, msg).partition(part).timestamp(ts).build() + } + testUtils.sendMessages(records) + } + + val firstTimestamp = System.currentTimeMillis() - 5000 + (0 to 2).foreach { partNum => + sendMessages(topic, (0 to 9).filter(_ % 3 == partNum) + .map(_.toString).toArray, partNum, firstTimestamp) + } + + val secondTimestamp = firstTimestamp + 1000 + (0 to 2).foreach { partNum => + sendMessages(topic, (10 to 19).filter(_ % 3 == partNum) + .map(_.toString).toArray, partNum, secondTimestamp) + } + + val thirdTimestamp = secondTimestamp + 1000 + (0 to 2).foreach { partNum => + sendMessages(topic, (20 to 29).filter(_ % 3 == partNum) + .map(_.toString).toArray, partNum, thirdTimestamp) + } + + val finalizedTimestamp = thirdTimestamp + 1000 + + (topic, Seq(firstTimestamp, secondTimestamp, thirdTimestamp, finalizedTimestamp)) + } + + private def verifyTimestampRelatedQueryResult( + optionFn: DataFrameReader => DataFrameReader, + topic: String, + expectation: Seq[Int]): Unit = { + val df = spark.read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + + val df2 = optionFn(df).load().selectExpr("CAST(value AS STRING)") + checkAnswer(df2, expectation.map(_.toString).toDF) + } + test("reuse same dataframe in query") { // This test ensures that we do not cache the Kafka Consumer in KafkaRelation val topic = newTopic() @@ -292,7 +480,8 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession }) } - private def testGroupId(groupIdKey: String, validateGroupId: (String, Iterable[String]) => Unit) { + private def testGroupId(groupIdKey: String, + validateGroupId: (String, Iterable[String]) => Unit): Unit = { // Tests code path KafkaSourceProvider.createRelation(.) val topic = newTopic() testUtils.createTopic(topic, partitions = 3) 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 fdda13b1bf6ec..d77b9a3b6a9e1 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 @@ -400,6 +400,7 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("topic", topic) + .mode("append") .save() checkAnswer( createKafkaReader(topic, includeHeaders = true).selectExpr( @@ -423,12 +424,13 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { df.write .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .mode("append") .save() } TestUtils.assertExceptionMsg(ex, "null topic present in the data") } - protected def testUnsupportedSaveModes(msg: (SaveMode) => String) { + protected def testUnsupportedSaveModes(msg: (SaveMode) => String): Unit = { val topic = newTopic() testUtils.createTopic(topic) val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") @@ -457,6 +459,7 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("topic", topic) + .mode("append") .save() } } 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 ee3eb43967069..bbb72bf9973e3 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 @@ -353,57 +353,33 @@ class KafkaTestUtils( } } - /** 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: _*)) + def sendMessages(topic: String, msgs: Array[String]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, msgs, None) } - /** 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]): Seq[(String, RecordMetadata)] = { - sendMessages(topic, messages, None) - } - - /** Send the array of messages to the Kafka broker using specified partition */ def sendMessages( topic: String, - messages: Array[String], - partition: Option[Int]): Seq[(String, RecordMetadata)] = { - sendMessages(topic, messages.map(m => (m, Seq())), partition) + msgs: Array[String], + part: Option[Int]): Seq[(String, RecordMetadata)] = { + val records = msgs.map { msg => + val builder = new RecordBuilder(topic, msg) + part.foreach { p => builder.partition(p) } + builder.build() + } + sendMessages(records) } - /** Send record to the Kafka broker with headers using specified partition */ - def sendMessage(topic: String, - record: (String, Seq[(String, Array[Byte])]), - partition: Option[Int]): Seq[(String, RecordMetadata)] = { - sendMessages(topic, Array(record).toSeq, partition) + def sendMessage(msg: ProducerRecord[String, String]): Seq[(String, RecordMetadata)] = { + sendMessages(Array(msg)) } - /** Send the array of records to the Kafka broker with headers using specified partition */ - def sendMessages(topic: String, - records: Seq[(String, Seq[(String, Array[Byte])])], - partition: Option[Int]): Seq[(String, RecordMetadata)] = { + def sendMessages(msgs: Seq[ProducerRecord[String, String]]): Seq[(String, RecordMetadata)] = { producer = new KafkaProducer[String, String](producerConfiguration) val offsets = try { - records.map { case (value, header) => - val headers = header.map { case (k, v) => - new RecordHeader(k, v).asInstanceOf[Header] - } - val record = partition match { - case Some(p) => - new ProducerRecord[String, String](topic, p, null, value, headers.asJava) - case None => - new ProducerRecord[String, String](topic, null, null, value, headers.asJava) - } - val metadata = producer.send(record).get(10, TimeUnit.SECONDS) - logInfo(s"\tSent ($value, $header) to partition ${metadata.partition}," + - " offset ${metadata.offset}") - (value, metadata) + msgs.map { msg => + val metadata = producer.send(msg).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent ($msg) to partition ${metadata.partition}, offset ${metadata.offset}") + (msg.value(), metadata) } } finally { if (producer != null) { @@ -574,7 +550,7 @@ class KafkaTestUtils( zkUtils: ZkUtils, topic: String, numPartitions: Int, - servers: Seq[KafkaServer]) { + servers: Seq[KafkaServer]): Unit = { eventually(timeout(1.minute), interval(200.milliseconds)) { try { verifyTopicDeletion(topic, numPartitions, servers) @@ -637,7 +613,7 @@ class KafkaTestUtils( val actualPort = factory.getLocalPort - def shutdown() { + def shutdown(): Unit = { 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 @@ -658,4 +634,3 @@ class KafkaTestUtils( } } } - diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.scala new file mode 100644 index 0000000000000..ef07798442e56 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.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.lang.{Integer => JInt, Long => JLong} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.header.Header +import org.apache.kafka.common.header.internals.RecordHeader + +class RecordBuilder(topic: String, value: String) { + var _partition: Option[JInt] = None + var _timestamp: Option[JLong] = None + var _key: Option[String] = None + var _headers: Option[Seq[(String, Array[Byte])]] = None + + def partition(part: JInt): RecordBuilder = { + _partition = Some(part) + this + } + + def partition(part: Int): RecordBuilder = { + _partition = Some(part.intValue()) + this + } + + def timestamp(ts: JLong): RecordBuilder = { + _timestamp = Some(ts) + this + } + + def timestamp(ts: Long): RecordBuilder = { + _timestamp = Some(ts.longValue()) + this + } + + def key(k: String): RecordBuilder = { + _key = Some(k) + this + } + + def headers(hdrs: Seq[(String, Array[Byte])]): RecordBuilder = { + _headers = Some(hdrs) + this + } + + def build(): ProducerRecord[String, String] = { + val part = _partition.orNull + val ts = _timestamp.orNull + val k = _key.orNull + val hdrs = _headers.map { h => + h.map { case (k, v) => new RecordHeader(k, v).asInstanceOf[Header] } + }.map(_.asJava).orNull + + new ProducerRecord[String, String](topic, part, ts, k, value, hdrs) + } +} 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 index 209b0d9035e7e..f54ff0d146f7a 100644 --- 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 @@ -57,6 +57,12 @@ private[spark] case class KafkaConfigUpdater(module: String, kafkaParams: Map[St } def setAuthenticationConfigIfNeeded(): this.type = { + val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(SparkEnv.get.conf, + kafkaParams(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) + setAuthenticationConfigIfNeeded(clusterConfig) + } + + def setAuthenticationConfigIfNeeded(clusterConfig: Option[KafkaTokenClusterConf]): 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'. @@ -66,8 +72,6 @@ private[spark] case class KafkaConfigUpdater(module: String, kafkaParams: Map[St 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.") setIfUnset(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, clusterConf.securityProtocol) 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 index 39e3ac74a9aeb..0ebe98330b4ae 100644 --- 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 @@ -36,7 +36,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, S 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.{SparkConf, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -241,8 +241,8 @@ private[spark] object KafkaTokenUtil extends Logging { "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, + REDACTION_REPLACEMENT_TEXT, tokenInfo.owner, tokenInfo.renewersAsString, dateFormat.format(tokenInfo.issueTimestamp), @@ -251,7 +251,7 @@ private[spark] object KafkaTokenUtil extends Logging { } } - def findMatchingToken( + def findMatchingTokenClusterConfig( sparkConf: SparkConf, bootStrapServers: String): Option[KafkaTokenClusterConf] = { val tokens = UserGroupInformation.getCurrentUser().getCredentials.getAllTokens.asScala @@ -272,6 +272,7 @@ private[spark] object KafkaTokenUtil extends Logging { def getTokenJaasParams(clusterConf: KafkaTokenClusterConf): String = { val token = UserGroupInformation.getCurrentUser().getCredentials.getToken( getTokenService(clusterConf.identifier)) + require(token != null, s"Token for identifier ${clusterConf.identifier} must exist") val username = new String(token.getIdentifier) val password = new String(token.getPassword) @@ -288,4 +289,17 @@ private[spark] object KafkaTokenUtil extends Logging { params } + + def isConnectorUsingCurrentToken( + params: ju.Map[String, Object], + clusterConfig: Option[KafkaTokenClusterConf]): Boolean = { + if (params.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) { + logDebug("Delegation token used by connector, checking if uses the latest token.") + val consumerJaasParams = params.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[String] + require(clusterConfig.isDefined, "Delegation token must exist for this connector.") + getTokenJaasParams(clusterConfig.get) == consumerJaasParams + } else { + true + } + } } 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 index 31815e93353d3..dc1e7cb8d979e 100644 --- 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 @@ -19,9 +19,11 @@ 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.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL} +import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_PLAINTEXT import org.apache.spark.SparkFunSuite @@ -65,14 +67,21 @@ class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTes } test("setAuthenticationConfigIfNeeded with global security should not set values") { - val params = Map.empty[String, String] + val params = Map( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers + ) + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers + ) + ) setGlobalKafkaClientConfig() val updatedParams = KafkaConfigUpdater(testModule, params) .setAuthenticationConfigIfNeeded() .build() - assert(updatedParams.size() === 0) + assert(updatedParams.asScala === params) } test("setAuthenticationConfigIfNeeded with token should set values") { @@ -98,13 +107,13 @@ class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTes } def testWithTokenSetValues(params: Map[String, String]) - (validate: (ju.Map[String, Object]) => Unit) { + (validate: (ju.Map[String, Object]) => Unit): Unit = { setSparkEnv( Map( s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers ) ) - addTokenToUGI(tokenService1) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) val updatedParams = KafkaConfigUpdater(testModule, params) .setAuthenticationConfigIfNeeded() @@ -128,7 +137,7 @@ class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTes s"spark.kafka.clusters.$identifier1.sasl.token.mechanism" -> "intentionally_invalid" ) ) - addTokenToUGI(tokenService1) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) val e = intercept[IllegalArgumentException] { KafkaConfigUpdater(testModule, params) 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 index eebbf96afa470..19335f4221e40 100644 --- 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 @@ -37,8 +37,12 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { 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 + private var savedSparkEnv: SparkEnv = _ + + protected val tokenId1 = "tokenId" + ju.UUID.randomUUID().toString + protected val tokenPassword1 = "tokenPassword" + ju.UUID.randomUUID().toString + protected val tokenId2 = "tokenId" + ju.UUID.randomUUID().toString + protected val tokenPassword2 = "tokenPassword" + ju.UUID.randomUUID().toString protected val identifier1 = "cluster1" protected val identifier2 = "cluster2" @@ -72,11 +76,16 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { } } + override def beforeEach(): Unit = { + super.beforeEach() + savedSparkEnv = SparkEnv.get + } + override def afterEach(): Unit = { try { Configuration.setConfiguration(null) - UserGroupInformation.setLoginUser(null) - SparkEnv.set(null) + UserGroupInformation.reset() + SparkEnv.set(savedSparkEnv) } finally { super.afterEach() } @@ -86,7 +95,7 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { Configuration.setConfiguration(new KafkaJaasConfiguration) } - protected def addTokenToUGI(tokenService: Text): Unit = { + protected def addTokenToUGI(tokenService: Text, tokenId: String, tokenPassword: String): Unit = { val token = new Token[KafkaDelegationTokenIdentifier]( tokenId.getBytes, tokenPassword.getBytes, 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 index 42a9fb5567b6f..225afbe5f3649 100644 --- 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 @@ -68,7 +68,7 @@ class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTes test("redactParams should redact token password from parameters") { setSparkEnv(Map.empty) val groupId = "id-" + ju.UUID.randomUUID().toString - addTokenToUGI(tokenService1) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) val clusterConf = createClusterConf(identifier1, SASL_SSL.name) val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) val kafkaParams = Seq( @@ -81,8 +81,8 @@ class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTes 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)) + assert(redactedJaasParams.contains(tokenId1)) + assert(!redactedJaasParams.contains(tokenPassword1)) } test("redactParams should redact passwords from parameters") { @@ -113,13 +113,13 @@ class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTes } test("redactJaasParam should redact token password") { - addTokenToUGI(tokenService1) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) 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)) + assert(redactedJaasParams.contains(tokenId1)) + assert(!redactedJaasParams.contains(tokenPassword1)) } } 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 index 5496195b41490..6fa1b56bff977 100644 --- 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 @@ -17,15 +17,18 @@ package org.apache.spark.kafka010 +import java.{util => ju} import java.security.PrivilegedExceptionAction +import scala.collection.JavaConverters._ + 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.{SparkConf, SparkEnv, SparkFunSuite} import org.apache.spark.internal.config._ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { @@ -174,58 +177,102 @@ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { assert(KafkaTokenUtil.isGlobalJaasConfigurationProvided) } - test("findMatchingToken without token should return None") { - assert(KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) === None) + test("findMatchingTokenClusterConfig without token should return None") { + assert(KafkaTokenUtil.findMatchingTokenClusterConfig(sparkConf, bootStrapServers) === None) } - test("findMatchingToken with non-matching tokens should return None") { + test("findMatchingTokenClusterConfig 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")) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) + addTokenToUGI(new Text("intentionally_garbage"), tokenId1, tokenPassword1) - assert(KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) === None) + assert(KafkaTokenUtil.findMatchingTokenClusterConfig(sparkConf, bootStrapServers) === None) } - test("findMatchingToken with one matching token should return cluster configuration") { + test("findMatchingTokenClusterConfig with one matching token should return token and 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) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) - assert(KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) === - Some(KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1))) + val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(sparkConf, bootStrapServers) + assert(clusterConfig.get === KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1)) } - test("findMatchingToken with multiple matching tokens should throw exception") { + test("findMatchingTokenClusterConfig 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) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) + addTokenToUGI(tokenService2, tokenId1, tokenPassword1) val thrown = intercept[IllegalArgumentException] { - KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) + KafkaTokenUtil.findMatchingTokenClusterConfig(sparkConf, bootStrapServers) } assert(thrown.getMessage.contains("More than one delegation token matches")) } test("getTokenJaasParams with token should return scram module") { - addTokenToUGI(tokenService1) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) 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)) + assert(jaasParams.contains(tokenId1)) + assert(jaasParams.contains(tokenPassword1)) + } + + test("isConnectorUsingCurrentToken without security should return true") { + val kafkaParams = Map[String, Object]().asJava + + assert(KafkaTokenUtil.isConnectorUsingCurrentToken(kafkaParams, None)) + } + + test("isConnectorUsingCurrentToken with same token should return true") { + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers + ) + ) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) + val kafkaParams = getKafkaParams() + val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(SparkEnv.get.conf, + kafkaParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) + + assert(KafkaTokenUtil.isConnectorUsingCurrentToken(kafkaParams, clusterConfig)) + } + + test("isConnectorUsingCurrentToken with different token should return false") { + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers + ) + ) + addTokenToUGI(tokenService1, tokenId1, tokenPassword1) + val kafkaParams = getKafkaParams() + addTokenToUGI(tokenService1, tokenId2, tokenPassword2) + val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(SparkEnv.get.conf, + kafkaParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) + + assert(!KafkaTokenUtil.isConnectorUsingCurrentToken(kafkaParams, clusterConfig)) + } + + private def getKafkaParams(): ju.Map[String, Object] = { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + Map[String, Object]( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers, + SaslConfigs.SASL_JAAS_CONFIG -> KafkaTokenUtil.getTokenJaasParams(clusterConf) + ).asJava } } 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 26b41e68efb49..925327d9d58e6 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 @@ -58,13 +58,13 @@ class DirectKafkaStreamSuite private var kafkaTestUtils: KafkaTestUtils = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } - override def afterAll() { + override def afterAll(): Unit = { try { if (kafkaTestUtils != null) { kafkaTestUtils.teardown() @@ -340,7 +340,7 @@ class DirectKafkaStreamSuite val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") // Send data to Kafka - def sendData(data: Seq[Int]) { + def sendData(data: Seq[Int]): Unit = { val strings = data.map { _.toString} kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) } @@ -432,7 +432,7 @@ class DirectKafkaStreamSuite val committed = new ConcurrentHashMap[TopicPartition, OffsetAndMetadata]() // Send data to Kafka and wait for it to be received - def sendDataAndWaitForReceive(data: Seq[Int]) { + def sendDataAndWaitForReceive(data: Seq[Int]): Unit = { val strings = data.map { _.toString} kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) eventually(timeout(10.seconds), interval(50.milliseconds)) { 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 431473e7f1d38..82913cf416a5f 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 @@ -27,7 +27,7 @@ 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.scalatestplus.mockito.MockitoSugar import org.apache.spark._ 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 47bc8fec2c80c..d6123e16dd238 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 @@ -47,14 +47,14 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private var sc: SparkContext = _ - override def beforeAll { + override def beforeAll: Unit = { super.beforeAll() sc = new SparkContext(sparkConf) kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } - override def afterAll { + override def afterAll: Unit = { try { try { if (sc != null) { @@ -81,7 +81,8 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private val preferredHosts = LocationStrategies.PreferConsistent - private def compactLogs(topic: String, partition: Int, messages: Array[(String, String)]) { + private def compactLogs(topic: String, partition: Int, + messages: Array[(String, String)]): Unit = { val mockTime = new MockTime() val logs = new Pool[TopicPartition, Log]() val logDir = kafkaTestUtils.brokerLogDir 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 5dec9709011e6..999870acfb532 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 @@ -316,7 +316,7 @@ private[kafka010] class KafkaTestUtils extends Logging { val actualPort = factory.getLocalPort - def shutdown() { + def shutdown(): Unit = { 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 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 dedd691cd1b23..d38ed9fc9263d 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 @@ -45,7 +45,7 @@ private[kafka010] class MockTime(@volatile private var currentMs: Long) extends override def nanoseconds: Long = TimeUnit.NANOSECONDS.convert(currentMs, TimeUnit.MILLISECONDS) - override def sleep(ms: Long) { + override def sleep(ms: Long): Unit = { this.currentMs += ms scheduler.tick() } diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index fcb790e3ea1f9..a5d5ac769b28d 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -73,7 +73,7 @@ import org.apache.spark.streaming.kinesis.KinesisInputDStream * the Kinesis Spark Streaming integration. */ object KinesisWordCountASL extends Logging { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { // Check that all required args were passed in. if (args.length != 3) { System.err.println( @@ -178,7 +178,7 @@ object KinesisWordCountASL extends Logging { * https://kinesis.us-east-1.amazonaws.com us-east-1 10 5 */ object KinesisWordProducerASL { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 4) { System.err.println( """ @@ -269,7 +269,7 @@ object KinesisWordProducerASL { */ private[streaming] object StreamingExamples extends Logging { // Set reasonable logging levels for streaming if the user has not configured log4j. - def setStreamingLogLevels() { + def setStreamingLogLevels(): Unit = { val log4jInitialized = Logger.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { // We first log something to initialize Spark's default logging, then we override the diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 9ea7d4081928b..6feb8f1b5598f 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -146,7 +146,7 @@ private[kinesis] class KinesisReceiver[T]( * This is called when the KinesisReceiver starts and must be non-blocking. * The KCL creates and manages the receiving/processing thread pool through Worker.run(). */ - override def onStart() { + override def onStart(): Unit = { blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) workerId = Utils.localHostName() + ":" + UUID.randomUUID() @@ -216,7 +216,7 @@ private[kinesis] class KinesisReceiver[T]( * The KCL worker.shutdown() method stops the receiving/processing threads. * The KCL will do its best to drain and checkpoint any in-flight records upon shutdown. */ - override def onStop() { + override def onStop(): Unit = { if (workerThread != null) { if (worker != null) { worker.shutdown() diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala index 8c6a399dd763e..b35573e92e168 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -51,7 +51,7 @@ private[kinesis] class KinesisRecordProcessor[T](receiver: KinesisReceiver[T], w * * @param shardId assigned by the KCL to this particular RecordProcessor. */ - override def initialize(shardId: String) { + override def initialize(shardId: String): Unit = { this.shardId = shardId logInfo(s"Initialized workerId $workerId with shardId $shardId") } @@ -65,7 +65,8 @@ private[kinesis] class KinesisRecordProcessor[T](receiver: KinesisReceiver[T], w * @param checkpointer used to update Kinesis when this batch has been processed/stored * in the DStream */ - override def processRecords(batch: List[Record], checkpointer: IRecordProcessorCheckpointer) { + override def processRecords(batch: List[Record], + checkpointer: IRecordProcessorCheckpointer): Unit = { if (!receiver.isStopped()) { try { // Limit the number of processed records from Kinesis stream. This is because the KCL cannot 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 ac0e6a8429d06..87592b6877b33 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 @@ -28,7 +28,7 @@ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} import org.apache.spark.util.ManualClock @@ -52,7 +52,7 @@ class KinesisCheckpointerSuite extends TestSuiteBase private var kinesisCheckpointer: KinesisCheckpointer = _ private var clock: ManualClock = _ - private val checkpoint = PrivateMethod[Unit]('checkpoint) + private val checkpoint = PrivateMethod[Unit](Symbol("checkpoint")) override def beforeEach(): Unit = { receiverMock = mock[KinesisReceiver[Array[Byte]]] diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala index 1c81298a7c201..8dc4de1aa3609 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala @@ -27,7 +27,7 @@ trait KinesisFunSuite extends SparkFunSuite { import KinesisTestUtils._ /** Run the test if environment variable is set or ignore the test */ - def testIfEnabled(testName: String)(testBody: => Unit) { + def testIfEnabled(testName: String)(testBody: => Unit): Unit = { if (shouldRunTests) { test(testName)(testBody) } else { diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala index 25357cb52edea..8b0d73c96da73 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala @@ -23,7 +23,7 @@ import collection.JavaConverters._ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration} import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel import org.scalatest.BeforeAndAfterEach -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Duration, Seconds, StreamingContext, TestSuiteBase} 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 52690847418ef..470a8cecc8fd9 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 @@ -27,7 +27,7 @@ import com.amazonaws.services.kinesis.model.Record 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 +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index ecc37dcaad1fe..d733868908350 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -81,13 +81,13 @@ object Edge { override def copyElement( src: Array[Edge[ED]], srcPos: Int, - dst: Array[Edge[ED]], dstPos: Int) { + dst: Array[Edge[ED]], dstPos: Int): Unit = { dst(dstPos) = src(srcPos) } override def copyRange( src: Array[Edge[ED]], srcPos: Int, - dst: Array[Edge[ED]], dstPos: Int, length: Int) { + dst: Array[Edge[ED]], dstPos: Int, length: Int): Unit = { System.arraycopy(src, srcPos, dst, dstPos, length) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala index ef0b943fc3c38..4ff5b02daecbe 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala @@ -30,7 +30,7 @@ object GraphXUtils { /** * Registers classes that GraphX uses with Kryo. */ - def registerKryoClasses(conf: SparkConf) { + def registerKryoClasses(conf: SparkConf): Unit = { conf.registerKryoClasses(Array( classOf[Edge[Object]], classOf[(VertexId, Object)], @@ -54,7 +54,7 @@ object GraphXUtils { mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { - def sendMsg(ctx: EdgeContext[VD, ED, A]) { + def sendMsg(ctx: EdgeContext[VD, ED, A]): Unit = { mapFunc(ctx.toEdgeTriplet).foreach { kv => val id = kv._1 val msg = kv._2 diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 0e6a340a680ba..8d03112a1c3dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -222,7 +222,7 @@ class EdgePartition[ * * @param f an external state mutating user defined function. */ - def foreach(f: Edge[ED] => Unit) { + def foreach(f: Edge[ED] => Unit): Unit = { iterator.foreach(f) } @@ -495,7 +495,7 @@ private class AggregatingEdgeContext[VD, ED, A]( srcId: VertexId, dstId: VertexId, localSrcId: Int, localDstId: Int, srcAttr: VD, dstAttr: VD, - attr: ED) { + attr: ED): Unit = { _srcId = srcId _dstId = dstId _localSrcId = localSrcId @@ -505,13 +505,13 @@ private class AggregatingEdgeContext[VD, ED, A]( _attr = attr } - def setSrcOnly(srcId: VertexId, localSrcId: Int, srcAttr: VD) { + def setSrcOnly(srcId: VertexId, localSrcId: Int, srcAttr: VD): Unit = { _srcId = srcId _localSrcId = localSrcId _srcAttr = srcAttr } - def setRest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED) { + def setRest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED): Unit = { _dstId = dstId _localDstId = localDstId _dstAttr = dstAttr @@ -524,14 +524,14 @@ private class AggregatingEdgeContext[VD, ED, A]( override def dstAttr: VD = _dstAttr override def attr: ED = _attr - override def sendToSrc(msg: A) { + override def sendToSrc(msg: A): Unit = { send(_localSrcId, msg) } - override def sendToDst(msg: A) { + override def sendToDst(msg: A): Unit = { send(_localDstId, msg) } - @inline private def send(localId: Int, msg: A) { + @inline private def send(localId: Int, msg: A): Unit = { if (bitset.get(localId)) { aggregates(localId) = mergeMsg(aggregates(localId), msg) } else { 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 27c08c894a39f..c7868f85d1f76 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 @@ -30,7 +30,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla private[this] val edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ - def add(src: VertexId, dst: VertexId, d: ED) { + def add(src: VertexId, dst: VertexId, d: ED): Unit = { edges += Edge(src, dst, d) } @@ -90,7 +90,7 @@ class ExistingEdgePartitionBuilder[ private[this] val edges = new PrimitiveVector[EdgeWithLocalIds[ED]](size) /** Add a new edge to the partition. */ - def add(src: VertexId, dst: VertexId, localSrc: Int, localDst: Int, d: ED) { + def add(src: VertexId, dst: VertexId, localSrc: Int, localDst: Int, d: ED): Unit = { edges += EdgeWithLocalIds(src, dst, localSrc, localDst, d) } @@ -153,13 +153,13 @@ private[impl] object EdgeWithLocalIds { override def copyElement( src: Array[EdgeWithLocalIds[ED]], srcPos: Int, - dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int): Unit = { dst(dstPos) = src(srcPos) } override def copyRange( src: Array[EdgeWithLocalIds[ED]], srcPos: Int, - dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int): Unit = { System.arraycopy(src, srcPos, dst, dstPos, length) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index d2194d85bf525..e0d4dd3248734 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -58,7 +58,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * `vertices`. This operation modifies the `ReplicatedVertexView`, and callers can access `edges` * afterwards to obtain the upgraded view. */ - def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, includeDst: Boolean) { + def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, includeDst: Boolean): Unit = { val shipSrc = includeSrc && !hasSrcId val shipDst = includeDst && !hasDstId if (shipSrc || shipDst) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 6453bbeae9f10..bef380dc12c23 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -123,7 +123,7 @@ class RoutingTablePartition( */ def foreachWithinEdgePartition (pid: PartitionID, includeSrc: Boolean, includeDst: Boolean) - (f: VertexId => Unit) { + (f: VertexId => Unit): Unit = { val (vidsCandidate, srcVids, dstVids) = routingTable(pid) val size = vidsCandidate.length if (includeSrc && includeDst) { 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 2847a4e172d40..c508056fe3ae3 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 @@ -98,7 +98,7 @@ object SVDPlusPlus { (ctx: EdgeContext[ (Array[Double], Array[Double], Double, Double), Double, - (Array[Double], Array[Double], Double)]) { + (Array[Double], Array[Double], Double)]): Unit = { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) val rank = p.length @@ -177,7 +177,7 @@ object SVDPlusPlus { // calculate error on training set def sendMsgTestF(conf: Conf, u: Double) - (ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]) { + (ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]): Unit = { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) var pred = u + usr._3 + itm._3 + blas.ddot(q.length, q, 1, usr._2, 1) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala index 2715137d19ebc..211b4d6e4c5d3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -85,7 +85,7 @@ object TriangleCount { } // Edge function computes intersection of smaller vertex with larger vertex - def edgeFunc(ctx: EdgeContext[VertexSet, ED, Int]) { + def edgeFunc(ctx: EdgeContext[VertexSet, ED, Int]): Unit = { val (smallSet, largeSet) = if (ctx.srcAttr.size < ctx.dstAttr.size) { (ctx.srcAttr, ctx.dstAttr) } else { 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 5ece5ae5c359b..dc3cdc452a389 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 @@ -118,7 +118,7 @@ private[graphx] object BytecodeUtils { if (name == methodName) { new MethodVisitor(ASM7) { override def visitMethodInsn( - op: Int, owner: String, name: String, desc: String, itf: Boolean) { + op: Int, owner: String, name: String, desc: String, itf: Boolean): Unit = { if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { if (!skipClass(owner)) { methodsInvoked.add((Utils.classForName(owner.replace("/", ".")), name)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 972237da1cb28..e3b283649cb2c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -71,7 +71,7 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } /** Set the value for a key */ - def update(k: K, v: V) { + def update(k: K, v: V): Unit = { val pos = keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK _values(pos) = v keySet.rehashIfNeeded(k, grow, move) @@ -80,7 +80,7 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, /** Set the value for a key */ - def setMerge(k: K, v: V, mergeF: (V, V) => V) { + def setMerge(k: K, v: V, mergeF: (V, V) => V): Unit = { val pos = keySet.addWithoutResize(k) val ind = pos & OpenHashSet.POSITION_MASK if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { // if first add 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 dede3b5d35ced..459cddb9a302b 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -164,12 +164,12 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { test("mapVertices changing type with same erased type") { withSpark { sc => - val vertices = sc.parallelize(Array[(Long, Option[java.lang.Integer])]( + val vertices = sc.parallelize(Seq[(Long, Option[java.lang.Integer])]( (1L, Some(1)), (2L, Some(2)), (3L, Some(3)) )) - val edges = sc.parallelize(Array( + val edges = sc.parallelize(Seq( Edge(1L, 2L, 0), Edge(2L, 3L, 0), Edge(3L, 1L, 0) @@ -218,8 +218,8 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { test("reverse with join elimination") { withSpark { sc => - val vertices: RDD[(VertexId, Int)] = sc.parallelize(Array((1L, 1), (2L, 2))) - val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0))) + val vertices: RDD[(VertexId, Int)] = sc.parallelize(Seq((1L, 1), (2L, 2))) + val edges: RDD[Edge[Int]] = sc.parallelize(Seq(Edge(1L, 2L, 0))) val graph = Graph(vertices, edges).reverse val result = GraphXUtils.mapReduceTriplets[Int, Int, Int]( graph, et => Iterator((et.dstId, et.srcAttr)), _ + _) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index 1b81423563372..baa1c42235c72 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -101,12 +101,12 @@ class ConnectedComponentsSuite extends SparkFunSuite with LocalSparkContext { withSpark { sc => // Create an RDD for the vertices val users: RDD[(VertexId, (String, String))] = - sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), + sc.parallelize(Seq((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), (4L, ("peter", "student")))) // Create an RDD for edges val relationships: RDD[Edge[String]] = - sc.parallelize(Array(Edge(3L, 7L, "collab"), Edge(5L, 3L, "advisor"), + sc.parallelize(Seq(Edge(3L, 7L, "collab"), Edge(5L, 3L, "advisor"), Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi"), Edge(4L, 0L, "student"), Edge(5L, 0L, "colleague"))) // Edges are: diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala index f19c3acdc85cf..abbd89b8eefaf 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala @@ -26,7 +26,7 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { test("Count a single triangle") { withSpark { sc => - val rawEdges = sc.parallelize(Array( 0L -> 1L, 1L -> 2L, 2L -> 0L ), 2) + val rawEdges = sc.parallelize(Seq(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = graph.triangleCount() val verts = triangleCount.vertices @@ -36,8 +36,8 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { test("Count two triangles") { withSpark { sc => - val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val triangles = Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = graph.triangleCount() @@ -55,8 +55,8 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { test("Count two triangles with bi-directed edges") { withSpark { sc => val triangles = - Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(0L -> -1L, -1L -> -2L, -2L -> 0L) val revTriangles = triangles.map { case (a, b) => (b, a) } val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() @@ -74,9 +74,9 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { test("Count a single triangle with duplicate edges") { withSpark { sc => - val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(1L -> 0L, 1L -> 1L), 2) + val rawEdges = sc.parallelize(Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(1L -> 0L, 1L -> 1L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() val triangleCount = graph.triangleCount() val verts = triangleCount.vertices diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 84940d96b563f..32844104c1deb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -26,8 +26,11 @@ import java.util.Map; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; +import java.util.logging.Level; +import java.util.logging.Logger; import static org.apache.spark.launcher.CommandBuilderUtils.*; +import static org.apache.spark.launcher.CommandBuilderUtils.join; /** * Launcher for Spark applications. @@ -38,6 +41,8 @@ */ public class SparkLauncher extends AbstractLauncher { + private static final Logger LOG = Logger.getLogger(SparkLauncher.class.getName()); + /** The Spark master. */ public static final String SPARK_MASTER = "spark.master"; @@ -363,6 +368,9 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr String loggerName = getLoggerName(); ProcessBuilder pb = createBuilder(); + if (LOG.isLoggable(Level.FINE)) { + LOG.fine(String.format("Launching Spark application:%n%s", join(" ", pb.command()))); + } boolean outputToLog = outputStream == null; boolean errorToLog = !redirectErrorStream && errorStream == null; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index f603e100fd2eb..383c3f60a595b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -390,9 +390,7 @@ boolean isClientMode(Map userProps) { String userMaster = firstNonEmpty(master, userProps.get(SparkLauncher.SPARK_MASTER)); String userDeployMode = firstNonEmpty(deployMode, userProps.get(SparkLauncher.DEPLOY_MODE)); // Default master is "local[*]", so assume client mode in that case - return userMaster == null || - "client".equals(userDeployMode) || - (!userMaster.equals("yarn") && userDeployMode == null); + return userMaster == null || userDeployMode == null || "client".equals(userDeployMode); } /** diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 32a91b1789412..752e8d4c23f8b 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -250,6 +250,26 @@ public void testMissingAppResource() { new SparkSubmitCommandBuilder().buildSparkSubmitArgs(); } + @Test + public void testIsClientMode() { + // Default master is "local[*]" + SparkSubmitCommandBuilder builder = newCommandBuilder(Collections.emptyList()); + assertTrue("By default application run in local mode", + builder.isClientMode(Collections.emptyMap())); + // --master yarn or it can be any RM + List sparkSubmitArgs = Arrays.asList(parser.MASTER, "yarn"); + builder = newCommandBuilder(sparkSubmitArgs); + assertTrue("By default deploy mode is client", builder.isClientMode(Collections.emptyMap())); + // --master yarn and set spark.submit.deployMode to client + Map userProps = new HashMap<>(); + userProps.put("spark.submit.deployMode", "client"); + assertTrue(builder.isClientMode(userProps)); + // --master mesos --deploy-mode cluster + sparkSubmitArgs = Arrays.asList(parser.MASTER, "mesos", parser.DEPLOY_MODE, "cluster"); + builder = newCommandBuilder(sparkSubmitArgs); + assertFalse(builder.isClientMode(Collections.emptyMap())); + } + private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) throws Exception { final String DRIVER_DEFAULT_PARAM = "-Ddriver-default"; final String DRIVER_EXTRA_PARAM = "-Ddriver-extra"; diff --git a/licenses-binary/LICENSE-dnsjava.txt b/licenses-binary/LICENSE-dnsjava.txt new file mode 100644 index 0000000000000..70ee5b12ff23f --- /dev/null +++ b/licenses-binary/LICENSE-dnsjava.txt @@ -0,0 +1,24 @@ +Copyright (c) 1998-2011, Brian Wellington. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/licenses-binary/LICENSE-javax-transaction-transaction-api.txt b/licenses-binary/LICENSE-javax-transaction-transaction-api.txt new file mode 100644 index 0000000000000..3d48d9c81fe12 --- /dev/null +++ b/licenses-binary/LICENSE-javax-transaction-transaction-api.txt @@ -0,0 +1,119 @@ +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 + +1. Definitions. + +1.1. Contributor means each individual or entity that creates or contributes to the creation of Modifications. + +1.2. Contributor Version means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + +1.3. Covered Software means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + +1.4. Executable means the Covered Software in any form other than Source Code. + +1.5. Initial Developer means the individual or entity that first makes Original Software available under this License. + +1.6. Larger Work means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + +1.7. License means this document. + +1.8. Licensable means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + +1.9. Modifications means the Source Code and Executable form of any of the following: + +A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + +B. Any new file that contains any part of the Original Software or previous Modification; or + +C. Any new file that is contributed or otherwise made available under the terms of this License. + +1.10. Original Software means the Source Code and Executable form of computer software code that is originally released under this License. + +1.11. Patent Claims means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + +1.12. Source Code means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + +1.13. You (or Your) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, You includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, control means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + +2.1. The Initial Developer Grant. +Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: +(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and +(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). +(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. +(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. +Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: +(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and +(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. + +Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + +3.2. Modifications. + +The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + +3.3. Required Notices. +You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + +3.5. Distribution of Executable Versions. +You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipients rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + +3.6. Larger Works. +You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. +Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + +4.2. Effect of New Versions. + +You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. +4.3. Modified Versions. + +When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + +6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as Participant) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + +6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a commercial item, as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as that term is defined at 48 C.F.R. 252.227-7014(a)(1)) and commercial computer software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdictions conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. + +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) +The GlassFish code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. + + + diff --git a/licenses-binary/LICENSE-jsp-api.txt b/licenses-binary/LICENSE-jsp-api.txt new file mode 100644 index 0000000000000..68076ad96b281 --- /dev/null +++ b/licenses-binary/LICENSE-jsp-api.txt @@ -0,0 +1,759 @@ +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.1 + +1. Definitions. + + 1.1. "Contributor" means each individual or entity that creates or + contributes to the creation of Modifications. + + 1.2. "Contributor Version" means the combination of the Original + Software, prior Modifications used by a Contributor (if any), and + the Modifications made by that particular Contributor. + + 1.3. "Covered Software" means (a) the Original Software, or (b) + Modifications, or (c) the combination of files containing Original + Software with files containing Modifications, in each case including + portions thereof. + + 1.4. "Executable" means the Covered Software in any form other than + Source Code. + + 1.5. "Initial Developer" means the individual or entity that first + makes Original Software available under this License. + + 1.6. "Larger Work" means a work which combines Covered Software or + portions thereof with code not governed by the terms of this License. + + 1.7. "License" means this document. + + 1.8. "Licensable" means having the right to grant, to the maximum + extent possible, whether at the time of the initial grant or + subsequently acquired, any and all of the rights conveyed herein. + + 1.9. "Modifications" means the Source Code and Executable form of + any of the following: + + A. Any file that results from an addition to, deletion from or + modification of the contents of a file containing Original Software + or previous Modifications; + + B. Any new file that contains any part of the Original Software or + previous Modification; or + + C. Any new file that is contributed or otherwise made available + under the terms of this License. + + 1.10. "Original Software" means the Source Code and Executable form + of computer software code that is originally released under this + License. + + 1.11. "Patent Claims" means any patent claim(s), now owned or + hereafter acquired, including without limitation, method, process, + and apparatus claims, in any patent Licensable by grantor. + + 1.12. "Source Code" means (a) the common form of computer software + code in which modifications are made and (b) associated + documentation included in or with such code. + + 1.13. "You" (or "Your") means an individual or a legal entity + exercising rights under, and complying with all of the terms of, + this License. For legal entities, "You" includes any entity which + controls, is controlled by, or is under common control with You. For + purposes of this definition, "control" means (a) the power, direct + or indirect, to cause the direction or management of such entity, + whether by contract or otherwise, or (b) ownership of more than + fifty percent (50%) of the outstanding shares or beneficial + ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, the Initial Developer + hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Initial Developer, to use, reproduce, + modify, display, perform, sublicense and distribute the Original + Software (or portions thereof), with or without Modifications, + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of + Original Software, to make, have made, use, practice, sell, and + offer for sale, and/or otherwise dispose of the Original Software + (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on + the date Initial Developer first distributes or otherwise makes the + Original Software available to a third party under the terms of this + License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is + granted: (1) for code that You delete from the Original Software, or + (2) for infringements caused by: (i) the modification of the + Original Software, or (ii) the combination of the Original Software + with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, each Contributor hereby + grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Contributor to use, reproduce, modify, + display, perform, sublicense and distribute the Modifications + created by such Contributor (or portions thereof), either on an + unmodified basis, with other Modifications, as Covered Software + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling + of Modifications made by that Contributor either alone and/or in + combination with its Contributor Version (or portions of such + combination), to make, use, sell, offer for sale, have made, and/or + otherwise dispose of: (1) Modifications made by that Contributor (or + portions thereof); and (2) the combination of Modifications made by + that Contributor with its Contributor Version (or portions of such + combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective + on the date Contributor first distributes or otherwise makes the + Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is + granted: (1) for any code that Contributor has deleted from the + Contributor Version; (2) for infringements caused by: (i) third + party modifications of Contributor Version, or (ii) the combination + of Modifications made by that Contributor with other software + (except as part of the Contributor Version) or other devices; or (3) + under Patent Claims infringed by Covered Software in the absence of + Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available + in Executable form must also be made available in Source Code form + and that Source Code form must be distributed only under the terms + of this License. You must include a copy of this License with every + copy of the Source Code form of the Covered Software You distribute + or otherwise make available. You must inform recipients of any such + Covered Software in Executable form as to how they can obtain such + Covered Software in Source Code form in a reasonable manner on or + through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are + governed by the terms of this License. You represent that You + believe Your Modifications are Your original creation(s) and/or You + have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that + identifies You as the Contributor of the Modification. You may not + remove or alter any copyright, patent or trademark notices contained + within the Covered Software, or any notices of licensing or any + descriptive text giving attribution to any Contributor or the + Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in + Source Code form that alters or restricts the applicable version of + this License or the recipients' rights hereunder. You may choose to + offer, and to charge a fee for, warranty, support, indemnity or + liability obligations to one or more recipients of Covered Software. + However, you may do so only on Your own behalf, and not on behalf of + the Initial Developer or any Contributor. You must make it + absolutely clear that any such warranty, support, indemnity or + liability obligation is offered by You alone, and You hereby agree + to indemnify the Initial Developer and every Contributor for any + liability incurred by the Initial Developer or such Contributor as a + result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under + the terms of this License or under the terms of a license of Your + choice, which may contain terms different from this License, + provided that You are in compliance with the terms of this License + and that the license for the Executable form does not attempt to + limit or alter the recipient's rights in the Source Code form from + the rights set forth in this License. If You distribute the Covered + Software in Executable form under a different license, You must make + it absolutely clear that any terms which differ from this License + are offered by You alone, not by the Initial Developer or + Contributor. You hereby agree to indemnify the Initial Developer and + every Contributor for any liability incurred by the Initial + Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with + other code not governed by the terms of this License and distribute + the Larger Work as a single product. In such a case, You must make + sure the requirements of this License are fulfilled for the Covered + Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or + new versions of this License from time to time. Each version will be + given a distinguishing version number. Except as provided in Section + 4.3, no one other than the license steward has the right to modify + this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the + Covered Software available under the terms of the version of the + License under which You originally received the Covered Software. If + the Initial Developer includes a notice in the Original Software + prohibiting it from being distributed or otherwise made available + under any subsequent version of the License, You must distribute and + make the Covered Software available under the terms of the version + of the License under which You originally received the Covered + Software. Otherwise, You may also choose to use, distribute or + otherwise make the Covered Software available under the terms of any + subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new + license for Your Original Software, You may create and use a + modified version of this License if You: (a) rename the license and + remove any references to the name of the license steward (except to + note that the license differs from this License); and (b) otherwise + make it clear that the license contains terms which differ from this + License. + +5. DISCLAIMER OF WARRANTY. + + COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, + WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, + INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE + IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR + NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF + THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE + DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY + OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, + REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN + ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS + AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate + automatically if You fail to comply with terms herein and fail to + cure such breach within 30 days of becoming aware of the breach. + Provisions which, by their nature, must remain in effect beyond the + termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding + declaratory judgment actions) against Initial Developer or a + Contributor (the Initial Developer or Contributor against whom You + assert such claim is referred to as "Participant") alleging that the + Participant Software (meaning the Contributor Version where the + Participant is a Contributor or the Original Software where the + Participant is the Initial Developer) directly or indirectly + infringes any patent, then any and all rights granted directly or + indirectly to You by such Participant, the Initial Developer (if the + Initial Developer is not the Participant) and all Contributors under + Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice + from Participant terminate prospectively and automatically at the + expiration of such 60 day notice period, unless if within such 60 + day period You withdraw Your claim with respect to the Participant + Software against such Participant either unilaterally or pursuant to + a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant + alleging that the Participant Software directly or indirectly + infringes any patent where such claim is resolved (such as by + license or settlement) prior to the initiation of patent + infringement litigation, then the reasonable value of the licenses + granted by such Participant under Sections 2.1 or 2.2 shall be taken + into account in determining the amount or value of any payment or + license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, + all end user licenses that have been validly granted by You or any + distributor hereunder prior to termination (excluding licenses + granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + + UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT + (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE + INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF + COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE + TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR + CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT + LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER + FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR + LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE + POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT + APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH + PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH + LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR + LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION + AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + + The Covered Software is a "commercial item," as that term is defined + in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer + software" (as that term is defined at 48 C.F.R. § + 252.227-7014(a)(1)) and "commercial computer software documentation" + as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent + with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 + (June 1995), all U.S. Government End Users acquire Covered Software + with only those rights set forth herein. This U.S. Government Rights + clause is in lieu of, and supersedes, any other FAR, DFAR, or other + clause or provision that addresses Government rights in computer + software under this License. + +9. MISCELLANEOUS. + + This License represents the complete agreement concerning subject + matter hereof. If any provision of this License is held to be + unenforceable, such provision shall be reformed only to the extent + necessary to make it enforceable. This License shall be governed by + the law of the jurisdiction specified in a notice contained within + the Original Software (except to the extent applicable law, if any, + provides otherwise), excluding such jurisdiction's conflict-of-law + provisions. Any litigation relating to this License shall be subject + to the jurisdiction of the courts located in the jurisdiction and + venue specified in a notice contained within the Original Software, + with the losing party responsible for costs, including, without + limitation, court costs and reasonable attorneys' fees and expenses. + The application of the United Nations Convention on Contracts for + the International Sale of Goods is expressly excluded. Any law or + regulation which provides that the language of a contract shall be + construed against the drafter shall not apply to this License. You + agree that You alone are responsible for compliance with the United + States export administration regulations (and the export control + laws and regulation of any other countries) when You use, distribute + or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + + As between Initial Developer and the Contributors, each party is + responsible for claims and damages arising, directly or indirectly, + out of its utilization of rights under this License and You agree to + work with Initial Developer and Contributors to distribute such + responsibility on an equitable basis. Nothing herein is intended or + shall be deemed to constitute any admission of liability. + +------------------------------------------------------------------------ + +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION +LICENSE (CDDL) + +The code released under the CDDL shall be governed by the laws of the +State of California (excluding conflict-of-law provisions). Any +litigation relating to this License shall be subject to the jurisdiction +of the Federal Courts of the Northern District of California and the +state courts of the State of California, with venue lying in Santa Clara +County, California. + + + + The GNU General Public License (GPL) Version 2, June 1991 + +Copyright (C) 1989, 1991 Free Software Foundation, Inc. +51 Franklin Street, Fifth Floor +Boston, MA 02110-1335 +USA + +Everyone is permitted to copy and distribute verbatim copies +of this license document, but changing it is not allowed. + +Preamble + +The licenses for most software are designed to take away your freedom to +share and change it. By contrast, the GNU General Public License is +intended to guarantee your freedom to share and change free software--to +make sure the software is free for all its users. This General Public +License applies to most of the Free Software Foundation's software and +to any other program whose authors commit to using it. (Some other Free +Software Foundation software is covered by the GNU Library General +Public License instead.) You can apply it to your programs, too. + +When we speak of free software, we are referring to freedom, not price. +Our General Public Licenses are designed to make sure that you have the +freedom to distribute copies of free software (and charge for this +service if you wish), that you receive source code or can get it if you +want it, that you can change the software or use pieces of it in new +free programs; and that you know you can do these things. + +To protect your rights, we need to make restrictions that forbid anyone +to deny you these rights or to ask you to surrender the rights. These +restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + +For example, if you distribute copies of such a program, whether gratis +or for a fee, you must give the recipients all the rights that you have. +You must make sure that they, too, receive or can get the source code. +And you must show them these terms so they know their rights. + +We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + +Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + +Finally, any free program is threatened constantly by software patents. +We wish to avoid the danger that redistributors of a free program will +individually obtain patent licenses, in effect making the program +proprietary. To prevent this, we have made it clear that any patent must +be licensed for everyone's free use or not licensed at all. + +The precise terms and conditions for copying, distribution and +modification follow. + +TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + +0. This License applies to any program or other work which contains a +notice placed by the copyright holder saying it may be distributed under +the terms of this General Public License. The "Program", below, refers +to any such program or work, and a "work based on the Program" means +either the Program or any derivative work under copyright law: that is +to say, a work containing the Program or a portion of it, either +verbatim or with modifications and/or translated into another language. +(Hereinafter, translation is included without limitation in the term +"modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of running +the Program is not restricted, and the output from the Program is +covered only if its contents constitute a work based on the Program +(independent of having been made by running the Program). Whether that +is true depends on what the Program does. + +1. You may copy and distribute verbatim copies of the Program's source +code as you receive it, in any medium, provided that you conspicuously +and appropriately publish on each copy an appropriate copyright notice +and disclaimer of warranty; keep intact all the notices that refer to +this License and to the absence of any warranty; and give any other +recipients of the Program a copy of this License along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + +2. You may modify your copy or copies of the Program or any portion of +it, thus forming a work based on the Program, and copy and distribute +such modifications or work under the terms of Section 1 above, provided +that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any part + thereof, to be licensed as a whole at no charge to all third parties + under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a notice + that there is no warranty (or else, saying that you provide a + warranty) and that users may redistribute the program under these + conditions, and telling the user how to view a copy of this License. + (Exception: if the Program itself is interactive but does not + normally print such an announcement, your work based on the Program + is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, and +can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based on +the Program, the distribution of the whole must be on the terms of this +License, whose permissions for other licensees extend to the entire +whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of a +storage or distribution medium does not bring the other work under the +scope of this License. + +3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your cost + of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed + only for noncommercial distribution and only if you received the + program in object code or executable form with such an offer, in + accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source code +means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to control +compilation and installation of the executable. However, as a special +exception, the source code distributed need not include anything that is +normally distributed (in either source or binary form) with the major +components (compiler, kernel, and so on) of the operating system on +which the executable runs, unless that component itself accompanies the +executable. + +If distribution of executable or object code is made by offering access +to copy from a designated place, then offering equivalent access to copy +the source code from the same place counts as distribution of the source +code, even though third parties are not compelled to copy the source +along with the object code. + +4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt otherwise +to copy, modify, sublicense or distribute the Program is void, and will +automatically terminate your rights under this License. However, parties +who have received copies, or rights, from you under this License will +not have their licenses terminated so long as such parties remain in +full compliance. + +5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and all +its terms and conditions for copying, distributing or modifying the +Program or works based on it. + +6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further restrictions +on the recipients' exercise of the rights granted herein. You are not +responsible for enforcing compliance by third parties to this License. + +7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot distribute +so as to satisfy simultaneously your obligations under this License and +any other pertinent obligations, then as a consequence you may not +distribute the Program at all. For example, if a patent license would +not permit royalty-free redistribution of the Program by all those who +receive copies directly or indirectly through you, then the only way you +could satisfy both it and this License would be to refrain entirely from +distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is implemented +by public license practices. Many people have made generous +contributions to the wide range of software distributed through that +system in reliance on consistent application of that system; it is up to +the author/donor to decide if he or she is willing to distribute +software through any other system and a licensee cannot impose that choice. + +This section is intended to make thoroughly clear what is believed to be +a consequence of the rest of this License. + +8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License may +add an explicit geographical distribution limitation excluding those +countries, so that distribution is permitted only in or among countries +not thus excluded. In such case, this License incorporates the +limitation as if written in the body of this License. + +9. The Free Software Foundation may publish revised and/or new +versions of the General Public License from time to time. Such new +versions will be similar in spirit to the present version, but may +differ in detail to address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and +conditions either of that version or of any later version published by +the Free Software Foundation. If the Program does not specify a version +number of this License, you may choose any version ever published by the +Free Software Foundation. + +10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the +author to ask for permission. For software which is copyrighted by the +Free Software Foundation, write to the Free Software Foundation; we +sometimes make exceptions for this. Our decision will be guided by the +two goals of preserving the free status of all derivatives of our free +software and of promoting the sharing and reuse of software generally. + +NO WARRANTY + +11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO +WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. +EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR +OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, +EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE +ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH +YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL +NECESSARY SERVICING, REPAIR OR CORRECTION. + +12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN +WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY +AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR +DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL +DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM +(INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED +INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF +THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR +OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +END OF TERMS AND CONDITIONS + +How to Apply These Terms to Your New Programs + +If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + +To do so, attach the following notices to the program. It is safest to +attach them to the start of each source file to most effectively convey +the exclusion of warranty; and each file should have at least the +"copyright" line and a pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, but + WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type + `show w'. This is free software, and you are welcome to redistribute + it under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the +appropriate parts of the General Public License. Of course, the commands +you use may be called something other than `show w' and `show c'; they +could even be mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + program `Gnomovision' (which makes passes at compilers) written by + James Hacker. + + signature of Ty Coon, 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program +into proprietary programs. If your program is a subroutine library, you +may consider it more useful to permit linking proprietary applications +with the library. If this is what you want to do, use the GNU Library +General Public License instead of this License. + +# + +Certain source files distributed by Oracle America, Inc. and/or its +affiliates are subject to the following clarification and special +exception to the GPLv2, based on the GNU Project exception for its +Classpath libraries, known as the GNU Classpath Exception, but only +where Oracle has expressly included in the particular source file's +header the words "Oracle designates this particular file as subject to +the "Classpath" exception as provided by Oracle in the LICENSE file +that accompanied this code." + +You should also note that Oracle includes multiple, independent +programs in this software package. Some of those programs are provided +under licenses deemed incompatible with the GPLv2 by the Free Software +Foundation and others. For example, the package includes programs +licensed under the Apache License, Version 2.0. Such programs are +licensed to you under their original licenses. + +Oracle facilitates your further distribution of this package by adding +the Classpath Exception to the necessary parts of its GPLv2 code, which +permits you to use that code in combination with other independent +modules not licensed under the GPLv2. However, note that this would +not permit you to commingle code under an incompatible license with +Oracle's GPLv2 licensed code by, for example, cutting and pasting such +code into a file also containing Oracle's GPLv2 licensed code and then +distributing the result. Additionally, if you were to remove the +Classpath Exception from any of the files to which it applies and +distribute the result, you would likely be required to license some or +all of the other code in that distribution under the GPLv2 as well, and +since the GPLv2 is incompatible with the license terms of some items +included in the distribution by Oracle, removing the Classpath +Exception could therefore effectively compromise your ability to +further distribute the package. + +Proceed with caution and we recommend that you obtain the advice of a +lawyer skilled in open source matters before removing the Classpath +Exception or making modifications to this package which may +subsequently be redistributed and/or involve the use of third party +software. + +CLASSPATH EXCEPTION +Linking this library statically or dynamically with other modules is +making a combined work based on this library. Thus, the terms and +conditions of the GNU General Public License version 2 cover the whole +combination. + +As a special exception, the copyright holders of this library give you +permission to link this library with independent modules to produce an +executable, regardless of the license terms of these independent +modules, and to copy and distribute the resulting executable under +terms of your choice, provided that you also meet, for each linked +independent module, the terms and conditions of the license of that +module. An independent module is a module which is not derived from or +based on this library. If you modify this library, you may extend this +exception to your version of the library, but you are not obligated to +do so. If you do not wish to do so, delete this exception statement +from your version. \ No newline at end of file diff --git a/licenses-binary/LICENSE-re2j.txt b/licenses-binary/LICENSE-re2j.txt new file mode 100644 index 0000000000000..0dc3cd70bf1f7 --- /dev/null +++ b/licenses-binary/LICENSE-re2j.txt @@ -0,0 +1,32 @@ +This is a work derived from Russ Cox's RE2 in Go, whose license +http://golang.org/LICENSE is as follows: + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala index 2a0f8c11d0a50..e054a15fc9b75 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala @@ -302,7 +302,7 @@ private[spark] object BLAS extends Serializable { * @param x the vector x that contains the n elements. * @param A the symmetric matrix A. Size of n x n. */ - def syr(alpha: Double, x: Vector, A: DenseMatrix) { + def syr(alpha: Double, x: Vector, A: DenseMatrix): Unit = { val mA = A.numRows val nA = A.numCols require(mA == nA, s"A is not a square matrix (and hence is not symmetric). A: $mA x $nA") @@ -316,7 +316,7 @@ private[spark] object BLAS extends Serializable { } } - private def syr(alpha: Double, x: DenseVector, A: DenseMatrix) { + private def syr(alpha: Double, x: DenseVector, A: DenseMatrix): Unit = { val nA = A.numRows val mA = A.numCols @@ -334,7 +334,7 @@ private[spark] object BLAS extends Serializable { } } - private def syr(alpha: Double, x: SparseVector, A: DenseMatrix) { + private def syr(alpha: Double, x: SparseVector, A: DenseMatrix): Unit = { val mA = A.numCols val xIndices = x.indices val xValues = x.values diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala index 332734bd28341..7d29d6dcea908 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala @@ -21,7 +21,7 @@ import java.util.Random import breeze.linalg.{CSCMatrix, Matrix => BM} import org.mockito.Mockito.when -import org.scalatest.mockito.MockitoSugar._ +import org.scalatestplus.mockito.MockitoSugar._ import scala.collection.mutable.{Map => MutableMap} import org.apache.spark.ml.SparkMLFunSuite diff --git a/mllib/benchmarks/UDTSerializationBenchmark-jdk11-results.txt b/mllib/benchmarks/UDTSerializationBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..6f671405b8343 --- /dev/null +++ b/mllib/benchmarks/UDTSerializationBenchmark-jdk11-results.txt @@ -0,0 +1,12 @@ +================================================================================================ +VectorUDT de/serialization +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +VectorUDT de/serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +serialize 269 292 13 0.0 269441.1 1.0X +deserialize 164 191 9 0.0 164314.6 1.6X + + diff --git a/mllib/benchmarks/UDTSerializationBenchmark-results.txt b/mllib/benchmarks/UDTSerializationBenchmark-results.txt index 169f4c60c748e..a0c853e99014b 100644 --- a/mllib/benchmarks/UDTSerializationBenchmark-results.txt +++ b/mllib/benchmarks/UDTSerializationBenchmark-results.txt @@ -2,12 +2,11 @@ VectorUDT de/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-6820HQ CPU @ 2.70GHz - -VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -serialize 144 / 206 0.0 143979.7 1.0X -deserialize 114 / 135 0.0 113802.6 1.3X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +VectorUDT de/serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +serialize 271 294 12 0.0 271054.3 1.0X +deserialize 190 192 2 0.0 189706.1 1.4X 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 58815434cbdaf..9eac8ed22a3f6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -62,6 +62,39 @@ private[ml] trait PredictorParams extends Params } SchemaUtils.appendColumn(schema, $(predictionCol), DoubleType) } + + /** + * Extract [[labelCol]], weightCol(if any) and [[featuresCol]] from the given dataset, + * and put it in an RDD with strong types. + */ + protected def extractInstances(dataset: Dataset[_]): RDD[Instance] = { + val w = this match { + case p: HasWeightCol => + if (isDefined(p.weightCol) && $(p.weightCol).nonEmpty) { + col($(p.weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + } + + dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } + } + + /** + * Extract [[labelCol]], weightCol(if any) and [[featuresCol]] from the given dataset, + * and put it in an RDD with strong types. + * Validate the output instances with the given function. + */ + protected def extractInstances(dataset: Dataset[_], + validateInstance: Instance => Unit): RDD[Instance] = { + extractInstances(dataset).map { instance => + validateInstance(instance) + instance + } + } } /** 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 b6b02e77909bd..9ac673078d4ad 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 @@ -20,7 +20,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException 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.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils} @@ -42,6 +42,22 @@ private[spark] trait ClassifierParams val parentSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) SchemaUtils.appendColumn(parentSchema, $(rawPredictionCol), new VectorUDT) } + + /** + * Extract [[labelCol]], weightCol(if any) and [[featuresCol]] from the given dataset, + * and put it in an RDD with strong types. + * Validates the label on the classifier is a valid integer in the range [0, numClasses). + */ + protected def extractInstances(dataset: Dataset[_], + numClasses: Int): RDD[Instance] = { + val validateInstance = (instance: Instance) => { + val label = instance.label + 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).") + } + extractInstances(dataset, validateInstance) + } } /** 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 6bd8a26f5b1a8..2d0212f36fad4 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,7 +22,7 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -34,9 +34,8 @@ 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, Row} -import org.apache.spark.sql.functions.{col, lit, udf} -import org.apache.spark.sql.types.DoubleType +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions.{col, udf} /** * Decision tree learning algorithm (http://en.wikipedia.org/wiki/Decision_tree_learning) @@ -116,9 +115,8 @@ class DecisionTreeClassifier @Since("1.4.0") ( dataset: Dataset[_]): DecisionTreeClassificationModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) - val categoricalFeatures: Map[Int, Int] = - MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val numClasses: Int = getNumClasses(dataset) + val categoricalFeatures = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val numClasses = getNumClasses(dataset) if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + @@ -126,13 +124,7 @@ class DecisionTreeClassifier @Since("1.4.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } 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 instances = extractInstances(dataset, numClasses) val strategy = getOldStrategy(categoricalFeatures, numClasses) instr.logNumClasses(numClasses) instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, 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 09f81b0dcbdae..74624be360c6a 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 @@ -203,6 +203,7 @@ class GBTClassifier @Since("1.4.0") ( } else { GradientBoostedTrees.run(trainDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) } + baseLearners.foreach(copyValues(_)) val numFeatures = baseLearners.head.numFeatures instr.logNumFeatures(numFeatures) 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 78503585261bf..e467228b4cc14 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 @@ -36,9 +36,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.{col, lit} /** Params for linear SVM Classifier. */ private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam @@ -161,12 +159,7 @@ class LinearSVC @Since("2.2.0") ( override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) override protected def train(dataset: Dataset[_]): LinearSVCModel = instrumented { instr => - val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - val instances: RDD[Instance] = - dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } + val instances = extractInstances(dataset) instr.logPipelineStage(this) instr.logDataset(dataset) 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 0997c1e7b38d6..af6e2b39ecb60 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 @@ -40,9 +40,8 @@ import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, Multiclas 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.functions.{col, lit} +import org.apache.spark.sql.functions.col import org.apache.spark.sql.types.{DataType, DoubleType, StructType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.VersionUtils @@ -492,12 +491,7 @@ class LogisticRegression @Since("1.2.0") ( protected[spark] def train( dataset: Dataset[_], handlePersistence: Boolean): LogisticRegressionModel = instrumented { instr => - val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - val instances: RDD[Instance] = - dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } + val instances = extractInstances(dataset) if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) 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 47b8a8df637b9..41db6f3f44342 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 @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.ml.ann.{FeedForwardTopology, FeedForwardTrainer} import org.apache.spark.ml.feature.OneHotEncoderModel -import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ 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 e97af0582d358..205f565aa2685 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.HasWeightCol @@ -28,7 +29,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.functions.col /** * Params for Naive Bayes Classifiers. @@ -137,17 +138,14 @@ class NaiveBayes @Since("1.5.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val modelTypeValue = $(modelType) - val requireValues: Vector => Unit = { - modelTypeValue match { - case Multinomial => - requireNonnegativeValues - case Bernoulli => - requireZeroOneBernoulliValues - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") - } + val validateInstance = $(modelType) match { + case Multinomial => + (instance: Instance) => requireNonnegativeValues(instance.features) + case Bernoulli => + (instance: Instance) => requireZeroOneBernoulliValues(instance.features) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, @@ -155,17 +153,15 @@ class NaiveBayes @Since("1.5.0") ( val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size instr.logNumFeatures(numFeatures) - val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) // Aggregates term frequencies per label. // TODO: Calling aggregateByKey and collect creates two stages, we can implement something // TODO: similar to reduceByKeyLocally to save one stage. - val aggregated = dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd - .map { row => (row.getDouble(0), (row.getDouble(1), row.getAs[Vector](2))) - }.aggregateByKey[(Double, DenseVector, Long)]((0.0, Vectors.zeros(numFeatures).toDense, 0L))( + val aggregated = extractInstances(dataset, validateInstance).map { instance => + (instance.label, (instance.weight, instance.features)) + }.aggregateByKey[(Double, DenseVector, Long)]((0.0, Vectors.zeros(numFeatures).toDense, 0L))( seqOp = { case ((weightSum, featureSum, count), (weight, features)) => - requireValues(features) BLAS.axpy(weight, features, featureSum) (weightSum + weight, featureSum, count + 1) }, 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 731b43b67813f..245cda35d8ade 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 @@ -143,6 +143,7 @@ class RandomForestClassifier @Since("1.4.0") ( val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) .map(_.asInstanceOf[DecisionTreeClassificationModel]) + trees.foreach(copyValues(_)) val numFeatures = trees.head.numFeatures instr.logNumClasses(numClasses) 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 2a7b3c579b078..09e8e7b232f3a 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 @@ -59,6 +59,28 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("1.2.0") def setMetricName(value: String): this.type = set(metricName, value) + /** + * param for number of bins to down-sample the curves (ROC curve, PR curve) in area + * computation. If 0, no down-sampling will occur. + * Default: 1000. + * @group expertParam + */ + @Since("3.0.0") + val numBins: IntParam = new IntParam(this, "numBins", "Number of bins to down-sample " + + "the curves (ROC curve, PR curve) in area computation. If 0, no down-sampling will occur. " + + "Must be >= 0.", + ParamValidators.gtEq(0)) + + /** @group expertGetParam */ + @Since("3.0.0") + def getNumBins: Int = $(numBins) + + /** @group expertSetParam */ + @Since("3.0.0") + def setNumBins(value: Int): this.type = set(numBins, value) + + setDefault(numBins -> 1000) + /** @group setParam */ @Since("1.5.0") def setRawPredictionCol(value: String): this.type = set(rawPredictionCol, value) @@ -94,7 +116,7 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va case Row(rawPrediction: Double, label: Double, weight: Double) => (rawPrediction, label, weight) } - val metrics = new BinaryClassificationMetrics(scoreAndLabelsWithWeights) + val metrics = new BinaryClassificationMetrics(scoreAndLabelsWithWeights, $(numBins)) val metric = $(metricName) match { case "areaUnderROC" => metrics.areaUnderROC() case "areaUnderPR" => metrics.areaUnderPR() @@ -104,10 +126,7 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va } @Since("1.5.0") - override def isLargerBetter: Boolean = $(metricName) match { - case "areaUnderROC" => true - case "areaUnderPR" => true - } + override def isLargerBetter: Boolean = true @Since("1.4.1") override def copy(extra: ParamMap): BinaryClassificationEvaluator = defaultCopy(extra) 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 85a6138c98a46..bac3e23774391 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,21 +18,23 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -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.ml.linalg.Vector +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.MulticlassMetrics import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType /** - * Evaluator for multiclass classification, which expects two input columns: prediction and label. + * Evaluator for multiclass classification, which expects input columns: prediction, label, + * weight (optional) and probability (only for logLoss). */ @Since("1.5.0") class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol - with HasWeightCol with DefaultParamsWritable { + extends Evaluator with HasPredictionCol with HasLabelCol with HasWeightCol + with HasProbabilityCol with DefaultParamsWritable { import MulticlassClassificationEvaluator.supportedMetricNames @@ -40,8 +42,12 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid def this() = this(Identifiable.randomUID("mcEval")) /** - * param for metric name in evaluation (supports `"f1"` (default), `"weightedPrecision"`, - * `"weightedRecall"`, `"accuracy"`) + * param for metric name in evaluation (supports `"f1"` (default), `"accuracy"`, + * `"weightedPrecision"`, `"weightedRecall"`, `"weightedTruePositiveRate"`, + * `"weightedFalsePositiveRate"`, `"weightedFMeasure"`, `"truePositiveRateByLabel"`, + * `"falsePositiveRateByLabel"`, `"precisionByLabel"`, `"recallByLabel"`, + * `"fMeasureByLabel"`, `"logLoss"`) + * * @group param */ @Since("1.5.0") @@ -71,6 +77,10 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) + /** @group setParam */ + @Since("3.0.0") + def setProbabilityCol(value: String): this.type = set(probabilityCol, value) + @Since("3.0.0") final val metricLabel: DoubleParam = new DoubleParam(this, "metricLabel", "The class whose metric will be computed in " + @@ -104,6 +114,21 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid setDefault(beta -> 1.0) + @Since("3.0.0") + final val eps: DoubleParam = new DoubleParam(this, "eps", + "log-loss is undefined for p=0 or p=1, so probabilities are clipped to " + + "max(eps, min(1 - eps, p)).", + ParamValidators.inRange(0, 0.5, false, false)) + + /** @group getParam */ + @Since("3.0.0") + def getEps: Double = $(eps) + + /** @group setParam */ + @Since("3.0.0") + def setEps(value: Double): this.type = set(eps, value) + + setDefault(eps -> 1e-15) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { @@ -111,13 +136,29 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid SchemaUtils.checkColumnType(schema, $(predictionCol), DoubleType) SchemaUtils.checkNumericType(schema, $(labelCol)) - val predictionAndLabelsWithWeights = - dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), - if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol))) + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + val rdd = if ($(metricName) == "logLoss") { + // probabilityCol is only needed to compute logloss + require(isDefined(probabilityCol) && $(probabilityCol).nonEmpty) + val p = DatasetUtils.columnToVector(dataset, $(probabilityCol)) + dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), w, p) + .rdd.map { + case Row(prediction: Double, label: Double, weight: Double, probability: Vector) => + (prediction, label, weight, probability.toArray) + } + } else { + dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), w) .rdd.map { case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) } - val metrics = new MulticlassMetrics(predictionAndLabelsWithWeights) + } + + val metrics = new MulticlassMetrics(rdd) $(metricName) match { case "f1" => metrics.weightedFMeasure case "accuracy" => metrics.accuracy @@ -131,16 +172,14 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "precisionByLabel" => metrics.precision($(metricLabel)) case "recallByLabel" => metrics.recall($(metricLabel)) case "fMeasureByLabel" => metrics.fMeasure($(metricLabel), $(beta)) + case "logLoss" => metrics.logLoss($(eps)) } } @Since("1.5.0") - override def isLargerBetter: Boolean = { - $(metricName) match { - case "weightedFalsePositiveRate" => false - case "falsePositiveRateByLabel" => false - case _ => true - } + override def isLargerBetter: Boolean = $(metricName) match { + case "weightedFalsePositiveRate" | "falsePositiveRateByLabel" | "logLoss" => false + case _ => true } @Since("1.5.0") @@ -154,7 +193,7 @@ object MulticlassClassificationEvaluator private val supportedMetricNames = Array("f1", "accuracy", "weightedPrecision", "weightedRecall", "weightedTruePositiveRate", "weightedFalsePositiveRate", "weightedFMeasure", "truePositiveRateByLabel", "falsePositiveRateByLabel", "precisionByLabel", "recallByLabel", - "fMeasureByLabel") + "fMeasureByLabel", "logLoss") @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/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index dd667a85fa598..b0cafefe420a3 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 @@ -18,7 +18,7 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.{BooleanParam, 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.RegressionMetrics @@ -43,13 +43,14 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui * - `"mse"`: mean squared error * - `"r2"`: R^2^ metric * - `"mae"`: mean absolute error + * - `"var"`: explained variance * * @group param */ @Since("1.4.0") val metricName: Param[String] = { - val allowedParams = ParamValidators.inArray(Array("mse", "rmse", "r2", "mae")) - new Param(this, "metricName", "metric name in evaluation (mse|rmse|r2|mae)", allowedParams) + val allowedParams = ParamValidators.inArray(Array("mse", "rmse", "r2", "mae", "var")) + new Param(this, "metricName", "metric name in evaluation (mse|rmse|r2|mae|var)", allowedParams) } /** @group getParam */ @@ -60,6 +61,25 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("1.4.0") def setMetricName(value: String): this.type = set(metricName, value) + /** + * param for whether the regression is through the origin. + * Default: false. + * @group expertParam + */ + @Since("3.0.0") + val throughOrigin: BooleanParam = new BooleanParam(this, "throughOrigin", + "Whether the regression is through the origin.") + + /** @group expertGetParam */ + @Since("3.0.0") + def getThroughOrigin: Boolean = $(throughOrigin) + + /** @group expertSetParam */ + @Since("3.0.0") + def setThroughOrigin(value: Boolean): this.type = set(throughOrigin, value) + + setDefault(throughOrigin -> false) + /** @group setParam */ @Since("1.4.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -86,22 +106,20 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui .rdd .map { case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) } - val metrics = new RegressionMetrics(predictionAndLabelsWithWeights) - val metric = $(metricName) match { + val metrics = new RegressionMetrics(predictionAndLabelsWithWeights, $(throughOrigin)) + $(metricName) match { case "rmse" => metrics.rootMeanSquaredError case "mse" => metrics.meanSquaredError case "r2" => metrics.r2 case "mae" => metrics.meanAbsoluteError + case "var" => metrics.explainedVariance } - metric } @Since("1.4.0") override def isLargerBetter: Boolean = $(metricName) match { - case "rmse" => false - case "mse" => false - case "r2" => true - case "mae" => false + case "r2" | "var" => true + case _ => false } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index c4daf64dfc5f0..ec4d45b65317f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -32,10 +32,17 @@ import org.apache.spark.sql.types._ /** * Binarize a column of continuous features given a threshold. + * + * Since 3.0.0, + * `Binarize` can map multiple columns at once by setting the `inputCols` parameter. Note that + * when both the `inputCol` and `inputCols` parameters are set, an Exception will be thrown. The + * `threshold` parameter is used for single column usage, and `thresholds` is for multiple + * columns. */ @Since("1.4.0") final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + extends Transformer with HasThreshold with HasThresholds with HasInputCol with HasOutputCol + with HasInputCols with HasOutputCols with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("binarizer")) @@ -48,19 +55,32 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) * @group param */ @Since("1.4.0") - val threshold: DoubleParam = + override val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold used to binarize continuous features") - /** @group getParam */ - @Since("1.4.0") - def getThreshold: Double = $(threshold) - /** @group setParam */ @Since("1.4.0") def setThreshold(value: Double): this.type = set(threshold, value) setDefault(threshold -> 0.0) + /** + * Array of threshold used to binarize continuous features. + * This is for multiple columns input. If transforming multiple columns and thresholds is + * not set, but threshold is set, then threshold will be applied across all columns. + * + * @group param + */ + @Since("3.0.0") + override val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Array of " + + "threshold used to binarize continuous features. This is for multiple columns input. " + + "If transforming multiple columns and thresholds is not set, but threshold is set, " + + "then threshold will be applied across all columns.") + + /** @group setParam */ + @Since("3.0.0") + def setThresholds(value: Array[Double]): this.type = set(thresholds, value) + /** @group setParam */ @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) @@ -69,66 +89,117 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) @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) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema, logging = true) - val schema = dataset.schema - val inputType = schema($(inputCol)).dataType - val td = $(threshold) - val metadata = outputSchema($(outputCol)).metadata - - val binarizerUDF = inputType match { - case DoubleType => - udf { in: Double => if (in > td) 1.0 else 0.0 } - - case _: VectorUDT if td >= 0 => - udf { vector: Vector => - val indices = ArrayBuilder.make[Int] - val values = ArrayBuilder.make[Double] - vector.foreachActive { (index, value) => - if (value > td) { - indices += index - values += 1.0 + + val (inputColNames, outputColNames, tds) = + if (isSet(inputCols)) { + if (isSet(thresholds)) { + ($(inputCols).toSeq, $(outputCols).toSeq, $(thresholds).toSeq) + } else { + ($(inputCols).toSeq, $(outputCols).toSeq, Seq.fill($(inputCols).length)($(threshold))) + } + } else { + (Seq($(inputCol)), Seq($(outputCol)), Seq($(threshold))) + } + + val ouputCols = inputColNames.zip(tds).map { case (inputColName, td) => + val binarizerUDF = dataset.schema(inputColName).dataType match { + case DoubleType => + udf { in: Double => if (in > td) 1.0 else 0.0 } + + case _: VectorUDT if td >= 0 => + udf { vector: Vector => + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + vector.foreachActive { (index, value) => + if (value > td) { + indices += index + values += 1.0 + } } + Vectors.sparse(vector.size, indices.result(), values.result()).compressed } - Vectors.sparse(vector.size, indices.result(), values.result()).compressed - } - case _: VectorUDT if td < 0 => - this.logWarning(s"Binarization operations on sparse dataset with negative threshold " + - s"$td will build a dense output, so take care when applying to sparse input.") - udf { vector: Vector => - val values = Array.fill(vector.size)(1.0) - vector.foreachActive { (index, value) => - if (value <= td) { - values(index) = 0.0 + case _: VectorUDT if td < 0 => + this.logWarning(s"Binarization operations on sparse dataset with negative threshold " + + s"$td will build a dense output, so take care when applying to sparse input.") + udf { vector: Vector => + val values = Array.fill(vector.size)(1.0) + vector.foreachActive { (index, value) => + if (value <= td) { + values(index) = 0.0 + } } + Vectors.dense(values).compressed } - Vectors.dense(values).compressed - } + } + + binarizerUDF(col(inputColName)) } - dataset.withColumn($(outputCol), binarizerUDF(col($(inputCol))), metadata) + val ouputMetadata = outputColNames.map(outputSchema(_).metadata) + dataset.withColumns(outputColNames, ouputCols, ouputMetadata) } @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - val outputColName = $(outputCol) - - val outCol: StructField = inputType match { - case DoubleType => - BinaryAttribute.defaultAttr.withName(outputColName).toStructField() - case _: VectorUDT => - StructField(outputColName, new VectorUDT) - case _ => - throw new IllegalArgumentException(s"Data type $inputType is not supported.") + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), + Seq(outputCols)) + + if (isSet(inputCol)) { + require(!isSet(thresholds), + s"thresholds can't be set for single-column Binarizer.") + } + + if (isSet(inputCols)) { + require(getInputCols.length == getOutputCols.length, + s"Binarizer $this has mismatched Params " + + s"for multi-column transform. Params (inputCols, outputCols) should have " + + s"equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}).") + if (isSet(thresholds)) { + require(getInputCols.length == getThresholds.length, + s"Binarizer $this has mismatched Params " + + s"for multi-column transform. Params (inputCols, outputCols, thresholds) " + + s"should have equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}, ${getThresholds.length}).") + require(!isSet(threshold), + s"exactly one of threshold, thresholds Params to be set, but both are set." ) + } + } + + val (inputColNames, outputColNames) = if (isSet(inputCols)) { + ($(inputCols).toSeq, $(outputCols).toSeq) + } else { + (Seq($(inputCol)), Seq($(outputCol))) } - if (schema.fieldNames.contains(outputColName)) { - throw new IllegalArgumentException(s"Output column $outputColName already exists.") + var outputFields = schema.fields + inputColNames.zip(outputColNames).foreach { case (inputColName, outputColName) => + require(!schema.fieldNames.contains(outputColName), + s"Output column $outputColName already exists.") + val inputType = schema(inputColName).dataType + val outputField = inputType match { + case DoubleType => + BinaryAttribute.defaultAttr.withName(outputColName).toStructField() + case _: VectorUDT => + StructField(outputColName, new VectorUDT) + case _ => + throw new IllegalArgumentException(s"Data type $inputType is not supported.") + } + outputFields :+= outputField } - StructType(schema.fields :+ outCol) + StructType(outputFields) } @Since("1.4.1") 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 16073d5fc1b6b..8533ed5ce5727 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 @@ -194,7 +194,7 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String if (isSet(inputCols)) { require(getInputCols.length == getOutputCols.length && getInputCols.length == getSplitsArray.length, s"Bucketizer $this has mismatched Params " + - s"for multi-column transform. Params (inputCols, outputCols, splitsArray) should have " + + s"for multi-column transform. Params (inputCols, outputCols, splitsArray) should have " + s"equal lengths, but they have different lengths: " + s"(${getInputCols.length}, ${getOutputCols.length}, ${getSplitsArray.length}).") @@ -283,7 +283,7 @@ object Bucketizer extends DefaultParamsReadable[Bucketizer] { val insertPos = -idx - 1 if (insertPos == 0 || insertPos == splits.length) { throw new SparkException(s"Feature value $feature out of Bucketizer bounds" + - s" [${splits.head}, ${splits.last}]. Check your features, or loosen " + + s" [${splits.head}, ${splits.last}]. Check your features, or loosen " + s"the lower/upper bound constraints.") } else { insertPos - 1 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 f7a83cdd41a90..aa4ab5903f711 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 @@ -180,13 +180,13 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui if (isSet(inputCols)) { require(getInputCols.length == getOutputCols.length, s"QuantileDiscretizer $this has mismatched Params " + - s"for multi-column transform. Params (inputCols, outputCols) should have " + + s"for multi-column transform. Params (inputCols, outputCols) should have " + s"equal lengths, but they have different lengths: " + s"(${getInputCols.length}, ${getOutputCols.length}).") if (isSet(numBucketsArray)) { require(getInputCols.length == getNumBucketsArray.length, s"QuantileDiscretizer $this has mismatched Params " + - s"for multi-column transform. Params (inputCols, outputCols, numBucketsArray) " + + s"for multi-column transform. Params (inputCols, outputCols, numBucketsArray) " + s"should have equal lengths, but they have different lengths: " + s"(${getInputCols.length}, ${getOutputCols.length}, ${getNumBucketsArray.length}).") require(!isSet(numBuckets), @@ -199,12 +199,12 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui } else { (Seq($(inputCol)), Seq($(outputCol))) } - val existingFields = schema.fields - var outputFields = existingFields + + var outputFields = schema.fields inputColNames.zip(outputColNames).foreach { case (inputColName, outputColName) => SchemaUtils.checkNumericType(schema, inputColName) - require(existingFields.forall(_.name != outputColName), - s"Output column ${outputColName} already exists.") + require(!schema.fieldNames.contains(outputColName), + s"Output column $outputColName already exists.") val attr = NominalAttribute.defaultAttr.withName(outputColName) outputFields :+= attr.toStructField() } 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 6c0d5fc70ab4e..9e95762f12914 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 @@ -30,6 +30,7 @@ import org.apache.spark.ml.param.{BooleanParam, Param, ParamMap, ParamValidators import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasHandleInvalid, HasLabelCol} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions.col import org.apache.spark.sql.types._ /** @@ -214,8 +215,11 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) col } + val terms = resolvedFormula.terms.flatten.distinct.sorted + lazy val firstRow = dataset.select(terms.map(col): _*).first() + // First we index each string column referenced by the input terms. - val indexed: Map[String, String] = resolvedFormula.terms.flatten.distinct.map { term => + val indexed = terms.zipWithIndex.map { case (term, i) => dataset.schema(term).dataType match { case _: StringType => val indexCol = tmpColumn("stridx") @@ -229,7 +233,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) case _: VectorUDT => val group = AttributeGroup.fromStructField(dataset.schema(term)) val size = if (group.size < 0) { - dataset.select(term).first().getAs[Vector](0).size + firstRow.getAs[Vector](i).size } else { group.size } @@ -392,7 +396,7 @@ class RFormulaModel private[feature]( } } - private def checkCanTransform(schema: StructType) { + private def checkCanTransform(schema: StructType): Unit = { val columnNames = schema.map(_.name) require(!columnNames.contains($(featuresCol)), "Features column already exists.") require( 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 eab90a32a3f4b..2ce5acf1fea13 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 @@ -31,7 +31,6 @@ 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.ThreadUtils import org.apache.spark.util.VersionUtils.majorMinorVersion import org.apache.spark.util.collection.OpenHashMap 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 f4abe9aa1c640..7ac9a288d285d 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 @@ -58,7 +58,7 @@ private[shared] object SharedParamsCodeGen { " The class with largest value p/t is predicted, where p is the original probability" + " of that class and t is the class's threshold", isValid = "(t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1", - finalMethods = false), + finalMethods = false, finalFields = false), ParamDesc[String]("inputCol", "input column name"), ParamDesc[Array[String]]("inputCols", "input column names"), ParamDesc[String]("outputCol", "output column name", Some("uid + \"__output\"")), diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 9baaf1744a3b7..6eeeb57e08fb0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -198,7 +198,7 @@ trait HasThresholds extends Params { * Param for Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold. * @group param */ - final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold", (t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1) + val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold", (t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1) /** @group getParam */ def getThresholds: Array[Double] = $(thresholds) 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 33097f322444b..5049ef924561c 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 @@ -131,7 +131,7 @@ private[recommendation] trait ALSModelParams extends Params with HasPredictionCo * Common params for ALS. */ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter with HasRegParam - with HasPredictionCol with HasCheckpointInterval with HasSeed { + with HasCheckpointInterval with HasSeed { /** * Param for rank of the matrix factorization (positive). @@ -801,7 +801,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square * matrix that it represents, storing it into destMatrix. */ - private def fillAtA(triAtA: Array[Double], lambda: Double) { + private def fillAtA(triAtA: Array[Double], lambda: Double): Unit = { var i = 0 var pos = 0 var a = 0.0 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 106be1b78af47..602b5fac20d3b 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.{Instance, LabeledPoint} +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._ @@ -34,9 +34,8 @@ 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.{Column, DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType /** @@ -118,12 +117,7 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S dataset: Dataset[_]): DecisionTreeRegressionModel = instrumented { instr => val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - 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 instances = extractInstances(dataset) val strategy = getOldStrategy(categoricalFeatures) instr.logPipelineStage(this) 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 00c0bc9f5e282..0cc06d82bf3fd 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 @@ -181,6 +181,7 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) GradientBoostedTrees.run(trainDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) } + baseLearners.foreach(copyValues(_)) val numFeatures = baseLearners.head.numFeatures instr.logNumFeatures(numFeatures) 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 abf75d70ea028..4c600eac26b37 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 @@ -43,7 +43,6 @@ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.regression.{LinearRegressionModel => OldLinearRegressionModel} 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, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -320,13 +319,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => // Extract the number of features before deciding optimization solver. val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size - val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - val instances: RDD[Instance] = dataset.select( - col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } + val instances = extractInstances(dataset) instr.logPipelineStage(this) instr.logDataset(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 938aa5acac086..8f78fc1da18c8 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 @@ -130,6 +130,7 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) .map(_.asInstanceOf[DecisionTreeRegressionModel]) + trees.foreach(copyValues(_)) val numFeatures = trees.head.numFeatures instr.logNamedValue(Instrumentation.loggerTags.numFeatures, numFeatures) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index c0a1683d3cb6f..314cf422be87e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -28,7 +28,8 @@ import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type */ -private[spark] abstract class Regressor[ +@DeveloperApi +abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], M <: RegressionModel[FeaturesType, M]] 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 8f8a17171f980..6c194902a750b 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 @@ -90,7 +90,7 @@ private[spark] class DecisionTreeMetadata( * Set number of splits for a continuous feature. * For a continuous feature, number of bins is number of splits plus 1. */ - def setNumSplits(featureIndex: Int, numSplits: Int) { + def setNumSplits(featureIndex: Int, numSplits: Int): Unit = { require(isContinuous(featureIndex), s"Only number of bin for a continuous feature can be set.") numBins(featureIndex) = numSplits + 1 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 8cd4a7ca9493b..58a763257af20 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 @@ -205,21 +205,21 @@ private[spark] class OptionalInstrumentation private( protected override def logName: String = className - override def logInfo(msg: => String) { + override def logInfo(msg: => String): Unit = { instrumentation match { case Some(instr) => instr.logInfo(msg) case None => super.logInfo(msg) } } - override def logWarning(msg: => String) { + override def logWarning(msg: => String): Unit = { instrumentation match { case Some(instr) => instr.logWarning(msg) case None => super.logWarning(msg) } } - override def logError(msg: => String) { + override def logError(msg: => String): Unit = { instrumentation match { case Some(instr) => instr.logError(msg) case None => super.logError(msg) 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 f6e51ab7f84f8..bafaafb720ed8 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 @@ -1311,7 +1311,7 @@ private[spark] abstract class SerDeBase { } } - private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit } def dumps(obj: AnyRef): Array[Byte] = { 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 ff4ca0ac40fe2..c7d44e8752cd9 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 @@ -269,7 +269,7 @@ class StreamingKMeans @Since("1.2.0") ( * @param data DStream containing vector data */ @Since("1.2.0") - def trainOn(data: DStream[Vector]) { + def trainOn(data: DStream[Vector]): Unit = { assertInitialized() data.foreachRDD { (rdd, time) => model = model.update(rdd, decayFactor, timeUnit) 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 d34a7ca6c9c7f..f4e2040569f48 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 @@ -81,7 +81,7 @@ class BinaryClassificationMetrics @Since("3.0.0") ( * Unpersist intermediate RDDs used in the computation. */ @Since("1.0.0") - def unpersist() { + def unpersist(): Unit = { cumulativeCounts.unpersist() } 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 c8245bf500a2f..9518f7e6828cf 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 @@ -23,33 +23,35 @@ 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, Row} +import org.apache.spark.sql.DataFrame /** * Evaluator for multiclass classification. * - * @param predictionAndLabels an RDD of (prediction, label, weight) or - * (prediction, label) tuples. + * @param predictionAndLabels an RDD of (prediction, label, weight, probability) or + * (prediction, label, weight) or (prediction, label) tuples. */ @Since("1.1.0") 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 + * @param predictionAndLabels a DataFrame with columns: prediction, label, weight (optional) + * and probability (only for logLoss) */ private[mllib] def this(predictionAndLabels: DataFrame) = - 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") + this(predictionAndLabels.rdd.map { r => + r.size match { + case 2 => (r.getDouble(0), r.getDouble(1), 1.0, null) + case 3 => (r.getDouble(0), r.getDouble(1), r.getDouble(2), null) + case 4 => (r.getDouble(0), r.getDouble(1), r.getDouble(2), r.getSeq[Double](3).toArray) + case _ => throw new IllegalArgumentException(s"Expected Row of tuples, got $r") + } }) - - private val confusions = predictionAndLabels.map { + private lazy val confusions = predictionAndLabels.map { + case (prediction: Double, label: Double, weight: Double, _) => + ((label, prediction), weight) case (prediction: Double, label: Double, weight: Double) => ((label, prediction), weight) case (prediction: Double, label: Double) => @@ -237,4 +239,38 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[_ <: Product]) */ @Since("1.1.0") lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted + + /** + * Returns the log-loss, aka logistic loss or cross-entropy loss. + * @param eps log-loss is undefined for p=0 or p=1, so probabilities are + * clipped to max(eps, min(1 - eps, p)). + */ + @Since("3.0.0") + def logLoss(eps: Double = 1e-15): Double = { + require(eps > 0 && eps < 0.5, s"eps must be in range (0, 0.5), but got $eps") + val loss1 = - math.log(eps) + val loss2 = - math.log1p(-eps) + + val (lossSum, weightSum) = predictionAndLabels.map { + case (_, label: Double, weight: Double, probability: Array[Double]) => + require(label.toInt == label && label >= 0, s"Invalid label $label") + require(probability != null, "probability of each class can not be null") + val p = probability(label.toInt) + val loss = if (p < eps) { + loss1 + } else if (p > 1 - eps) { + loss2 + } else { + - math.log(p) + } + (loss * weight, weight) + + case other => + throw new IllegalArgumentException(s"Expected quadruples, got $other") + }.treeReduce { case ((l1, w1), (l2, w2)) => + (l1 + l2, w1 + w2) + } + + lossSum / weightSum + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index cb97742245689..1f5558dc2a50e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -285,7 +285,7 @@ private[spark] object BLAS extends Serializable with Logging { * @param x the vector x that contains the n elements. * @param A the symmetric matrix A. Size of n x n. */ - def syr(alpha: Double, x: Vector, A: DenseMatrix) { + def syr(alpha: Double, x: Vector, A: DenseMatrix): Unit = { val mA = A.numRows val nA = A.numCols require(mA == nA, s"A is not a square matrix (and hence is not symmetric). A: $mA x $nA") @@ -299,7 +299,7 @@ private[spark] object BLAS extends Serializable with Logging { } } - private def syr(alpha: Double, x: DenseVector, A: DenseMatrix) { + private def syr(alpha: Double, x: DenseVector, A: DenseMatrix): Unit = { val nA = A.numRows val mA = A.numCols @@ -317,7 +317,7 @@ private[spark] object BLAS extends Serializable with Logging { } } - private def syr(alpha: Double, x: SparseVector, A: DenseMatrix) { + private def syr(alpha: Double, x: SparseVector, A: DenseMatrix): Unit = { val mA = A.numCols val xIndices = x.indices val xValues = x.values 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 e474cfa002fad..0304fd88dcd9f 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 @@ -155,7 +155,7 @@ sealed trait Matrix extends Serializable { * and column indices respectively with the type `Int`, and the final parameter is the * corresponding value in the matrix with type `Double`. */ - private[spark] def foreachActive(f: (Int, Int, Double) => Unit) + private[spark] def foreachActive(f: (Int, Int, Double) => Unit): Unit /** * Find the number of non-zero active values. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 0d223de9b6f7e..f3b984948e483 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -153,7 +153,7 @@ class CoordinateMatrix @Since("1.0.0") ( } /** Determines the size by computing the max row/column index. */ - private def computeSize() { + private def computeSize(): Unit = { // Reduce will throw an exception if `entries` is empty. val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => (math.max(i1, i2), math.max(j1, j2)) 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 43f48befd014f..f25d86b30631a 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 @@ -770,7 +770,7 @@ class RowMatrix @Since("1.0.0") ( } /** Updates or verifies the number of rows. */ - private def updateNumRows(m: Long) { + private def updateNumRows(m: Long): Unit = { if (nRows <= 0) { nRows = m } else { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index fa04f8eb5e796..d3b548832bb21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -107,7 +107,7 @@ class PoissonGenerator @Since("1.1.0") ( override def nextValue(): Double = rng.sample() @Since("1.1.0") - override def setSeed(seed: Long) { + override def setSeed(seed: Long): Unit = { rng.reseedRandomGenerator(seed) } @@ -132,7 +132,7 @@ class ExponentialGenerator @Since("1.3.0") ( override def nextValue(): Double = rng.sample() @Since("1.3.0") - override def setSeed(seed: Long) { + override def setSeed(seed: Long): Unit = { rng.reseedRandomGenerator(seed) } @@ -159,7 +159,7 @@ class GammaGenerator @Since("1.3.0") ( override def nextValue(): Double = rng.sample() @Since("1.3.0") - override def setSeed(seed: Long) { + override def setSeed(seed: Long): Unit = { rng.reseedRandomGenerator(seed) } @@ -187,7 +187,7 @@ class LogNormalGenerator @Since("1.3.0") ( override def nextValue(): Double = rng.sample() @Since("1.3.0") - override def setSeed(seed: Long) { + override def setSeed(seed: Long): Unit = { rng.reseedRandomGenerator(seed) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 7f84be9f37822..b6eb10e9de00a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -65,7 +65,7 @@ object KMeansDataGenerator { } @Since("0.8.0") - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 6) { // scalastyle:off println println("Usage: KMeansGenerator " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 58fd010e4905f..c218681b3375e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -189,7 +189,7 @@ object LinearDataGenerator { } @Since("0.8.0") - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { // scalastyle:off println println("Usage: LinearDataGenerator " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 68835bc79677f..7e9d9465441c9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -65,7 +65,7 @@ object LogisticRegressionDataGenerator { } @Since("0.8.0") - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length != 5) { // scalastyle:off println println("Usage: LogisticRegressionGenerator " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 42c5bcdd39f76..7a308a5ec25c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -54,7 +54,7 @@ import org.apache.spark.rdd.RDD @Since("0.8.0") object MFDataGenerator { @Since("0.8.0") - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { // scalastyle:off println println("Usage: MFDataGenerator " + 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 6d15a6bb01e4e..9198334ba02a1 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 @@ -173,7 +173,7 @@ object MLUtils extends Logging { * @see `org.apache.spark.mllib.util.MLUtils.loadLibSVMFile` */ @Since("1.0.0") - def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { + def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String): Unit = { // TODO: allow to specify label precision and feature precision. val dataStr = data.map { case LabeledPoint(label, features) => val sb = new StringBuilder(label.toString) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index c9468606544db..9f6ba025aedde 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -37,7 +37,7 @@ import org.apache.spark.rdd.RDD object SVMDataGenerator { @Since("0.8.0") - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { if (args.length < 2) { // scalastyle:off println println("Usage: SVMGenerator " + diff --git a/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala index e2ee7c05ab399..f2343b7a88560 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala @@ -25,7 +25,7 @@ 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.scalatestplus.mockito.MockitoSugar.mock import org.apache.spark.SparkFunSuite import org.apache.spark.ml.param.ParamMap 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 1183cb0617610..e6025a5a53ca6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.when -import org.scalatest.mockito.MockitoSugar.mock +import org.scalatestplus.mockito.MockitoSugar.mock import org.apache.spark.SparkFunSuite import org.apache.spark.ml.Pipeline.SharedReadWrite 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 9f2053dcc91fc..3ebf8a83a892c 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 @@ -44,7 +44,7 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { private val seed = 42 - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() categoricalDataPointsRDD = sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()).map(_.asML) 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 467f13f808a01..530ca20d0eb0c 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 @@ -55,7 +55,7 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { private val eps: Double = 1e-5 private val absEps: Double = 1e-8 - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) .map(_.asML) @@ -456,6 +456,22 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { } } + test("tree params") { + val categoricalFeatures = Map.empty[Int, Int] + val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) + val gbt = new GBTClassifier() + .setMaxDepth(2) + .setCheckpointInterval(5) + .setSeed(123) + val model = gbt.fit(df) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getCheckpointInterval === model.getCheckpointInterval) + assert(i.getSeed === model.getSeed) + }) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// 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 0f0954e5d8cac..5958bfcf5ea6d 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 @@ -42,7 +42,7 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ private var orderedLabeledPoints5_20: RDD[LabeledPoint] = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) @@ -56,7 +56,7 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { // Tests calling train() ///////////////////////////////////////////////////////////////////////////// - def binaryClassificationTestWithContinuousFeatures(rf: RandomForestClassifier) { + def binaryClassificationTestWithContinuousFeatures(rf: RandomForestClassifier): Unit = { val categoricalFeatures = Map.empty[Int, Int] val numClasses = 2 val newRF = rf @@ -230,6 +230,26 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { } } + test("tree params") { + val rdd = orderedLabeledPoints5_20 + val rf = new RandomForestClassifier() + .setImpurity("entropy") + .setMaxDepth(3) + .setNumTrees(3) + .setSeed(123) + val categoricalFeatures = Map.empty[Int, Int] + val numClasses = 2 + + val df: DataFrame = TreeTests.setMetadata(rdd, categoricalFeatures, numClasses) + val model = rf.fit(df) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getSeed === model.getSeed) + assert(i.getImpurity === model.getImpurity) + }) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// 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 b41df1b798af6..5b5212abdf7cc 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 @@ -18,6 +18,7 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -60,4 +61,23 @@ class MulticlassClassificationEvaluatorSuite .setMetricLabel(1.0) assert(evaluator.evaluate(predictionAndLabels) ~== 3.0 / 4 absTol 1e-5) } + + test("MulticlassClassificationEvaluator support logloss") { + val labels = Seq(1.0, 2.0, 0.0, 1.0) + val probabilities = Seq( + Vectors.dense(0.1, 0.8, 0.1), + Vectors.dense(0.9, 0.05, 0.05), + Vectors.dense(0.8, 0.2, 0.0), + Vectors.dense(0.3, 0.65, 0.05)) + + val df = sc.parallelize(labels.zip(probabilities)).map { + case (label, probability) => + val prediction = probability.argmax.toDouble + (prediction, label, probability) + }.toDF("prediction", "label", "probability") + + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("logLoss") + assert(evaluator.evaluate(df) ~== 0.9682005730687164 absTol 1e-5) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala index c1a156959618e..f4f858c3e92dc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala @@ -76,6 +76,10 @@ class RegressionEvaluatorSuite // mae evaluator.setMetricName("mae") assert(evaluator.evaluate(predictions) ~== 0.08399089 absTol 0.01) + + // var + evaluator.setMetricName("var") + assert(evaluator.evaluate(predictions) ~== 63.6944519 absTol 0.01) } test("read/write") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index 91bec50fb904f..9baad52db00b3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -122,5 +122,131 @@ class BinarizerSuite extends MLTest with DefaultReadWriteTest { .setOutputCol("myOutputCol") .setThreshold(0.1) testDefaultReadWrite(t) + + val t2 = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setThresholds(Array(30.0, 30.0, 30.0)) + testDefaultReadWrite(t2) + } + + test("Multiple Columns: Test thresholds") { + val thresholds = Array(10.0, -0.5, 0.0) + + val data1 = Seq(5.0, 11.0) + val expected1 = Seq(0.0, 1.0) + val data2 = Seq(Vectors.sparse(3, Array(1), Array(0.5)), + Vectors.dense(Array(0.0, 0.5, 0.0))) + val expected2 = Seq(Vectors.dense(Array(1.0, 1.0, 1.0)), + Vectors.dense(Array(1.0, 1.0, 1.0))) + val data3 = Seq(0.0, 1.0) + val expected3 = Seq(0.0, 1.0) + + val df = Seq(0, 1).map { idx => + (data1(idx), data2(idx), data3(idx), expected1(idx), expected2(idx), expected3(idx)) + }.toDF("input1", "input2", "input3", "expected1", "expected2", "expected3") + + val binarizer = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setThresholds(thresholds) + + binarizer.transform(df) + .select("result1", "expected1", "result2", "expected2", "result3", "expected3") + .collect().foreach { + case Row(r1: Double, e1: Double, r2: Vector, e2: Vector, r3: Double, e3: Double) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + assert(r3 === e3, + s"The result value is not correct after bucketing. Expected $e3 but found $r3") + } + } + + test("Multiple Columns: Comparing setting threshold with setting thresholds " + + "explicitly with identical values") { + val data1 = Array.range(1, 21, 1).map(_.toDouble) + val data2 = Array.range(1, 40, 2).map(_.toDouble) + val data3 = Array.range(1, 60, 3).map(_.toDouble) + val df = (0 until 20).map { idx => + (data1(idx), data2(idx), data3(idx)) + }.toDF("input1", "input2", "input3") + + val binarizerSingleThreshold = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setThreshold(30.0) + + val df2 = binarizerSingleThreshold.transform(df) + + val binarizerMultiThreshold = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("expected1", "expected2", "expected3")) + .setThresholds(Array(30.0, 30.0, 30.0)) + + binarizerMultiThreshold.transform(df2) + .select("result1", "expected1", "result2", "expected2", "result3", "expected3") + .collect().foreach { + case Row(r1: Double, e1: Double, r2: Double, e2: Double, r3: Double, e3: Double) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + assert(r3 === e3, + s"The result value is not correct after bucketing. Expected $e3 but found $r3") + } + } + + test("Multiple Columns: Mismatched sizes of inputCols/outputCols") { + val binarizer = new Binarizer() + .setInputCols(Array("input")) + .setOutputCols(Array("result1", "result2")) + .setThreshold(1.0) + val df = sc.parallelize(Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) + .map(Tuple1.apply).toDF("input") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } + } + + test("Multiple Columns: Mismatched sizes of inputCols/thresholds") { + val binarizer = new Binarizer() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setThresholds(Array(1.0, 2.0, 3.0)) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } + } + + test("Multiple Columns: Mismatched sizes of inputCol/thresholds") { + val binarizer = new Binarizer() + .setInputCol("input1") + .setOutputCol("result1") + .setThresholds(Array(1.0, 2.0)) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } + } + + test("Multiple Columns: Set both of threshold/thresholds") { + val binarizer = new Binarizer() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setThresholds(Array(1.0, 2.0)) + .setThreshold(1.0) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } } } 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 add1cc17ea057..efd56f7073a19 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 @@ -25,7 +25,7 @@ class RFormulaParserSuite extends SparkFunSuite { formula: String, label: String, terms: Seq[String], - schema: StructType = new StructType) { + schema: StructType = new StructType): Unit = { val resolved = RFormulaParser.parse(formula).resolve(schema) assert(resolved.label == label) val simpleTerms = terms.map { t => 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 630e785e59507..49ebcb385640e 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 @@ -40,7 +40,7 @@ class DecisionTreeRegressorSuite extends MLTest with DefaultReadWriteTest { private val seed = 42 - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() categoricalDataPointsRDD = sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints().map(_.asML)) 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 884fe2d11bf5a..e2462af2ac1d1 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 @@ -47,7 +47,7 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { private var trainData: RDD[LabeledPoint] = _ private var validationData: RDD[LabeledPoint] = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) .map(_.asML) @@ -296,7 +296,21 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { } } - ///////////////////////////////////////////////////////////////////////////// + test("tree params") { + val gbt = new GBTRegressor() + .setMaxDepth(2) + .setCheckpointInterval(5) + .setSeed(123) + val model = gbt.fit(trainData.toDF) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getCheckpointInterval === model.getCheckpointInterval) + assert(i.getSeed === model.getSeed) + }) + } + + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index c6dabd1b28829..f3b0f0470e579 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -38,7 +38,7 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) @@ -49,7 +49,7 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ // Tests calling train() ///////////////////////////////////////////////////////////////////////////// - def regressionTestWithContinuousFeatures(rf: RandomForestRegressor) { + def regressionTestWithContinuousFeatures(rf: RandomForestRegressor): Unit = { val categoricalFeaturesInfo = Map.empty[Int, Int] val newRF = rf .setImpurity("variance") @@ -139,6 +139,25 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ } } + test("tree params") { + val rf = new RandomForestRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(10) + .setNumTrees(3) + .setSeed(123) + + val df = orderedLabeledPoints50_1000.toDF() + val model = rf.fit(df) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getSeed === model.getSeed) + assert(i.getImpurity === model.getImpurity) + assert(i.getMaxBins === model.getMaxBins) + }) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// 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 a63ab913f2c22..ae5e979983b4f 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 @@ -485,7 +485,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } } - def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: OldStrategy) { + def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures( + strategy: OldStrategy): Unit = { val numFeatures = 50 val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) val rdd = sc.parallelize(arr).map(_.asML.toInstance) 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 8a0a48ff6095b..978a3cbe54c1e 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 @@ -56,7 +56,7 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => sc.setCheckpointDir(checkpointDir) } - override def afterAll() { + override def afterAll(): Unit = { try { Utils.deleteRecursively(new File(checkpointDir)) } finally { @@ -127,17 +127,17 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => dataframe: DataFrame, transformer: Transformer, expectedMessagePart : String, - firstResultCol: String) { + firstResultCol: String): Unit = { withClue(s"""Expected message part "${expectedMessagePart}" is not found in DF test.""") { val exceptionOnDf = intercept[Throwable] { - testTransformerOnDF(dataframe, transformer, firstResultCol)(_ => Unit) + testTransformerOnDF(dataframe, transformer, firstResultCol)(_ => ()) } 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) + testTransformerOnStreamData(dataframe, transformer, firstResultCol)(_ => ()) } TestUtils.assertExceptionMsg(exceptionOnStreamData, expectedMessagePart) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 126f0a23bfcb9..d4e9da3c6263e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -206,7 +206,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w def validatePrediction( predictions: Seq[Double], input: Seq[LabeledPoint], - expectedAcc: Double = 0.83) { + expectedAcc: Double = 0.83): Unit = { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label } 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 725389813b3e2..47dac3ec29a5c 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 @@ -91,7 +91,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { import NaiveBayes.{Multinomial, Bernoulli} - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]): Unit = { val numOfPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 3676d9c5debc8..007b8ae6e1a6a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -62,7 +62,7 @@ object SVMSuite { class SVMSuite extends SparkFunSuite with MLlibTestSparkContext { - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]): Unit = { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label } 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 8779de590a256..e10295c905cdb 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 @@ -176,4 +176,82 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { (weight0 * f2measure0 + weight1 * f2measure1 + weight2 * f2measure2) relTol delta) assert(metrics.labels === labels) } + + test("MulticlassMetrics supports binary class log-loss") { + /* + Using the following Python code to verify the correctness. + + from sklearn.metrics import log_loss + labels = [1, 0, 0, 1] + probabilities = [[.1, .9], [.9, .1], [.8, .2], [.35, .65]] + weights = [1.5, 2.0, 1.0, 0.5] + + >>> log_loss(y_true=labels, y_pred=probabilities, sample_weight=weights) + 0.16145936283256573 + >>> log_loss(y_true=labels, y_pred=probabilities) + 0.21616187468057912 + */ + + val labels = Seq(1.0, 0.0, 0.0, 1.0) + val probabilities = Seq( + Array(0.1, 0.9), + Array(0.9, 0.1), + Array(0.8, 0.2), + Array(0.35, 0.65)) + val weights = Seq(1.5, 2.0, 1.0, 0.5) + + val rdd = sc.parallelize(labels.zip(weights).zip(probabilities)).map { + case ((label, weight), probability) => + val prediction = probability.indexOf(probability.max).toDouble + (prediction, label, weight, probability) + } + val metrics = new MulticlassMetrics(rdd) + assert(metrics.logLoss() ~== 0.16145936283256573 relTol delta) + + val rdd2 = rdd.map { + case (prediction: Double, label: Double, weight: Double, probability: Array[Double]) => + (prediction, label, 1.0, probability) + } + val metrics2 = new MulticlassMetrics(rdd2) + assert(metrics2.logLoss() ~== 0.21616187468057912 relTol delta) + } + + test("MulticlassMetrics supports multi-class log-loss") { + /* + Using the following Python code to verify the correctness. + + from sklearn.metrics import log_loss + labels = [1, 2, 0, 1] + probabilities = [[.1, .8, .1], [.9, .05, .05], [.8, .2, .0], [.3, .65, .05]] + weights = [1.5, 2.0, 1.0, 0.5] + + >>> log_loss(y_true=labels, y_pred=probabilities, sample_weight=weights) + 1.3529429766879466 + >>> log_loss(y_true=labels, y_pred=probabilities) + 0.9682005730687164 + */ + + val labels = Seq(1.0, 2.0, 0.0, 1.0) + val probabilities = Seq( + Array(0.1, 0.8, 0.1), + Array(0.9, 0.05, 0.05), + Array(0.8, 0.2, 0.0), + Array(0.3, 0.65, 0.05)) + val weights = Seq(1.5, 2.0, 1.0, 0.5) + + val rdd = sc.parallelize(labels.zip(weights).zip(probabilities)).map { + case ((label, weight), probability) => + val prediction = probability.indexOf(probability.max).toDouble + (prediction, label, weight, probability) + } + val metrics = new MulticlassMetrics(rdd) + assert(metrics.logLoss() ~== 1.3529429766879466 relTol delta) + + val rdd2 = rdd.map { + case (prediction: Double, label: Double, weight: Double, probability: Array[Double]) => + (prediction, label, 1.0, probability) + } + val metrics2 = new MulticlassMetrics(rdd2) + assert(metrics2.logLoss() ~== 0.9682005730687164 relTol delta) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala index 305cb4cbbdeea..4bc84b83ab1db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala @@ -189,6 +189,6 @@ object ChiSqSelectorSuite extends SparkFunSuite { } def checkEqual(a: ChiSqSelectorModel, b: ChiSqSelectorModel): Unit = { - assert(a.selectedFeatures.deep == b.selectedFeatures.deep) + assert(a.selectedFeatures.sameElements(b.selectedFeatures)) } } 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 b4520d42fedf5..184c89c9eaaf9 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 @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import breeze.linalg.{CSCMatrix, Matrix => BM} import org.mockito.Mockito.when -import org.scalatest.mockito.MockitoSugar._ +import org.scalatestplus.mockito.MockitoSugar._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.Kryo._ @@ -39,7 +39,7 @@ class MatricesSuite extends SparkFunSuite { val ser = new KryoSerializer(conf).newInstance() - def check[T: ClassTag](t: T) { + def check[T: ClassTag](t: T): Unit = { assert(ser.deserialize[T](ser.serialize(t)) === t) } 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 b2163b518dbd1..c0c5c5c7d98d5 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 @@ -42,7 +42,7 @@ class VectorsSuite extends SparkFunSuite with Logging { conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() - def check[T: ClassTag](t: T) { + def check[T: ClassTag](t: T): Unit = { assert(ser.deserialize[T](ser.serialize(t)) === t) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index f6a996940291c..9d7177e0a149e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -35,7 +35,7 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val numPartitions = 3 var gridBasedMat: BlockMatrix = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() val blocks: Seq[((Int, Int), Matrix)] = Seq( ((0, 0), new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala index 37d75103d18d2..d197f06a393e8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -29,7 +29,7 @@ class CoordinateMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val n = 4 var mat: CoordinateMatrix = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() val entries = sc.parallelize(Seq( (0, 0, 1.0), 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 cca4eb4e4260e..e961d10711860 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 @@ -36,7 +36,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { ).map(x => IndexedRow(x._1, x._2)) var indexedRows: RDD[IndexedRow] = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() indexedRows = sc.parallelize(data, 2) } 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 a0c4c68243e67..0a4b11935580a 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 @@ -57,7 +57,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { var denseMat: RowMatrix = _ var sparseMat: RowMatrix = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() denseMat = new RowMatrix(sc.parallelize(denseData, 2)) sparseMat = new RowMatrix(sc.parallelize(sparseData, 2)) @@ -213,7 +213,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { brzNorm(v, 1.0) < 1e-6 } - def assertColumnEqualUpToSign(A: BDM[Double], B: BDM[Double], k: Int) { + def assertColumnEqualUpToSign(A: BDM[Double], B: BDM[Double], k: Int): Unit = { assert(A.rows === B.rows) for (j <- 0 until k) { val aj = A(::, j) @@ -338,7 +338,7 @@ class RowMatrixClusterSuite extends SparkFunSuite with LocalClusterSparkContext var mat: RowMatrix = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() val m = 4 val n = 200000 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 b3bf5a2a8f2cc..a629c6951abcd 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 @@ -25,7 +25,7 @@ import org.apache.spark.util.StatCounter class RandomDataGeneratorSuite extends SparkFunSuite { - def apiChecks(gen: RandomDataGenerator[Double]) { + def apiChecks(gen: RandomDataGenerator[Double]): Unit = { // resetting seed should generate the same sequence of random numbers gen.setSeed(42L) val array1 = (0 until 1000).map(_ => gen.nextValue()) @@ -56,7 +56,7 @@ class RandomDataGeneratorSuite extends SparkFunSuite { def distributionChecks(gen: RandomDataGenerator[Double], mean: Double = 0.0, stddev: Double = 1.0, - epsilon: Double = 0.01) { + epsilon: Double = 0.01): Unit = { for (seed <- 0 until 5) { gen.setSeed(seed.toLong) val sample = (0 until 100000).map { _ => gen.nextValue()} 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 9b4dc29d326a1..470e1016dab39 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 @@ -38,7 +38,7 @@ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Seri expectedNumPartitions: Int, expectedMean: Double, expectedStddev: Double, - epsilon: Double = 0.01) { + epsilon: Double = 0.01): Unit = { val stats = rdd.stats() assert(expectedSize === stats.count) assert(expectedNumPartitions === rdd.partitions.size) @@ -53,7 +53,7 @@ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Seri expectedNumPartitions: Int, expectedMean: Double, expectedStddev: Double, - epsilon: Double = 0.01) { + epsilon: Double = 0.01): Unit = { assert(expectedNumPartitions === rdd.partitions.size) val values = new ArrayBuffer[Double]() rdd.collect.foreach { vector => { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala index 56231429859ee..db17db9c8597d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext class MLPairRDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext { test("topByKey") { - val topMap = sc.parallelize(Array((1, 7), (1, 3), (1, 6), (1, 1), (1, 2), (3, 2), (3, 7), (5, + val topMap = sc.parallelize(Seq((1, 7), (1, 3), (1, 6), (1, 1), (1, 2), (3, 2), (3, 7), (5, 1), (3, 5)), 2) .topByKey(5) .collectAsMap() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index b08ad99f4f204..9be87db873dad 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -224,7 +224,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { negativeWeights: Boolean = false, numUserBlocks: Int = -1, numProductBlocks: Int = -1, - negativeFactors: Boolean = true) { + negativeFactors: Boolean = true): Unit = { // scalastyle:on val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index b4281d66ec1f8..f336dac0ccb5d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -33,7 +33,7 @@ private object LassoSuite { class LassoSuite extends SparkFunSuite with MLlibTestSparkContext { - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]): Unit = { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected.label) > 0.5 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index a9765b007b450..be0834d0fd7df 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -33,7 +33,7 @@ private object LinearRegressionSuite { class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]): Unit = { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected.label) > 0.5 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 5b94f7ea4a075..8e2d7d10f2ce2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -34,13 +34,13 @@ class StreamingLinearRegressionSuite override def maxWaitTimeMillis: Int = 20000 // Assert that two values are equal within tolerance epsilon - def assertEqual(v1: Double, v2: Double, epsilon: Double) { + def assertEqual(v1: Double, v2: Double, epsilon: Double): Unit = { def errorMessage = v1.toString + " did not equal " + v2.toString assert(math.abs(v1-v2) <= epsilon, errorMessage) } // Assert that model predictions are correct - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]): Unit = { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected.label) > 0.5 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 88b9d4c039ba9..b738236473230 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 @@ -437,7 +437,7 @@ object DecisionTreeSuite extends SparkFunSuite { def validateClassifier( model: DecisionTreeModel, input: Seq[LabeledPoint], - requiredAccuracy: Double) { + requiredAccuracy: Double): Unit = { val predictions = input.map(x => model.predict(x.features)) val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label @@ -450,7 +450,7 @@ object DecisionTreeSuite extends SparkFunSuite { def validateRegressor( model: DecisionTreeModel, input: Seq[LabeledPoint], - requiredMSE: Double) { + requiredMSE: Double): Unit = { val predictions = input.map(x => model.predict(x.features)) val squaredError = predictions.zip(input).map { case (prediction, expected) => val err = prediction - expected.label 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 d43e62bb65535..e04d7b7c327a8 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 @@ -37,7 +37,7 @@ object EnsembleTestHelper { numCols: Int, expectedMean: Double, expectedStddev: Double, - epsilon: Double) { + epsilon: Double): Unit = { val values = new mutable.ArrayBuffer[Double]() data.foreach { row => assert(row.size == numCols) @@ -51,7 +51,7 @@ object EnsembleTestHelper { def validateClassifier( model: TreeEnsembleModel, input: Seq[LabeledPoint], - requiredAccuracy: Double) { + requiredAccuracy: Double): Unit = { val predictions = input.map(x => model.predict(x.features)) val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label @@ -68,7 +68,7 @@ object EnsembleTestHelper { model: TreeEnsembleModel, input: Seq[LabeledPoint], required: Double, - metricName: String = "mse") { + metricName: String = "mse"): Unit = { val predictions = input.map(x => model.predict(x.features)) val errors = predictions.zip(input).map { case (prediction, point) => point.label - prediction diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index bec61ba6a003c..b1a385a576cea 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * Test suite for [[RandomForest]]. */ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { - def binaryClassificationTestWithContinuousFeatures(strategy: Strategy) { + def binaryClassificationTestWithContinuousFeatures(strategy: Strategy): Unit = { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val numTrees = 1 @@ -68,7 +68,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { binaryClassificationTestWithContinuousFeatures(strategy) } - def regressionTestWithContinuousFeatures(strategy: Strategy) { + def regressionTestWithContinuousFeatures(strategy: Strategy): Unit = { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val numTrees = 1 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 2853b752cb85c..79d4785fd6fa7 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 @@ -25,7 +25,7 @@ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() val conf = new SparkConf() .setMaster("local-cluster[2, 1, 1024]") @@ -34,7 +34,7 @@ trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => sc = new SparkContext(conf) } - override def afterAll() { + override def afterAll(): Unit = { try { if (sc != null) { sc.stop() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala index 720237bd2dddd..f9a3cd088314e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala @@ -31,7 +31,7 @@ trait MLlibTestSparkContext extends TempDirectory { self: Suite => @transient var sc: SparkContext = _ @transient var checkpointDir: String = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() spark = SparkSession.builder .master("local[2]") @@ -43,7 +43,7 @@ trait MLlibTestSparkContext extends TempDirectory { self: Suite => sc.setCheckpointDir(checkpointDir) } - override def afterAll() { + override def afterAll(): Unit = { try { Utils.deleteRecursively(new File(checkpointDir)) SparkSession.clearActiveSession() diff --git a/pom.xml b/pom.xml index 2396c5168b166..c42ef5c6626d1 100644 --- a/pom.xml +++ b/pom.xml @@ -125,7 +125,7 @@ 2.7.4 2.5.0 ${hadoop.version} - 3.4.6 + 3.4.14 2.7.1 0.4.2 org.spark-project.hive @@ -148,7 +148,7 @@ 0.9.3 2.4.0 2.0.8 - 3.1.5 + 3.2.6 1.8.2 hadoop2 1.8.10 @@ -170,8 +170,7 @@ true 1.9.13 - 2.9.9 - 2.9.9.3 + 2.10.0 1.1.7.3 1.1.2 1.10 @@ -488,7 +487,7 @@ commons-beanutils commons-beanutils - 1.9.3 + 1.9.4 commons-logging @@ -620,7 +619,7 @@ com.github.luben zstd-jni - 1.4.2-1 + 1.4.3-1 com.clearspring.analytics @@ -658,7 +657,7 @@ io.netty netty-all - 4.1.39.Final + 4.1.42.Final org.apache.derby @@ -698,7 +697,7 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.jackson.databind.version} + ${fasterxml.jackson.version} com.fasterxml.jackson.core @@ -833,7 +832,7 @@ org.scala-lang.modules scala-parser-combinators_${scala.binary.version} - 1.1.0 + 1.1.2 jline @@ -843,7 +842,7 @@ org.scalatest scalatest_${scala.binary.version} - 3.0.5 + 3.0.8 test @@ -861,7 +860,7 @@ org.scalacheck scalacheck_${scala.binary.version} - 1.13.5 + 1.14.2 test @@ -1337,6 +1336,10 @@ io.netty netty + + com.github.spotbugs + spotbugs-annotations + @@ -1996,75 +1999,6 @@ - - ${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 @@ -2245,6 +2179,11 @@ + + com.univocity + univocity-parsers + 2.8.3 + @@ -2302,7 +2241,7 @@ net.alchim31.maven scala-maven-plugin - 4.2.0 + 4.2.4 eclipse-add-source @@ -2341,7 +2280,6 @@ -deprecation -feature -explaintypes - -Yno-adapted-args -target:jvm-1.8 @@ -2361,7 +2299,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.8.0 + 3.8.1 ${java.version} ${java.version} @@ -2524,7 +2462,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.0.1 + 3.1.1 -Xdoclint:all @@ -2805,7 +2743,7 @@ org.apache.maven.plugins maven-checkstyle-plugin - 3.0.0 + 3.1.0 false true @@ -2825,7 +2763,7 @@ com.puppycrawl.tools checkstyle - 8.23 + 8.25 @@ -2979,7 +2917,6 @@ 3.2.0 2.13.0 - 3.4.13 org.apache.hive core ${hive23.version} @@ -3058,6 +2995,19 @@ scala-2.12 + + + scala-2.13 + + + + org.scala-lang.modules + scala-parallel-collections_${scala.binary.version} + 0.2.0 + + + + - - com.squareup.okhttp3 - okhttp - 3.8.1 - - org.mockito mockito-core 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 54afe92e81567..819ea4a7b13d0 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 @@ -142,6 +142,12 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_EXECUTOR_SCHEDULER_NAME = + ConfigBuilder("spark.kubernetes.executor.scheduler.name") + .doc("Specify the scheduler name for each executor pod") + .stringConf + .createOptional + val KUBERNETES_EXECUTOR_REQUEST_CORES = ConfigBuilder("spark.kubernetes.executor.request.cores") .doc("Specify the cpu request for each executor pod") 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 a2a46614fb8f8..09943b7974ed9 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 @@ -24,6 +24,7 @@ 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.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.util.Utils @@ -123,7 +124,7 @@ private[spark] class KubernetesExecutorConf( val appId: String, val executorId: String, val driverPod: Option[Pod]) - extends KubernetesConf(sparkConf) { + extends KubernetesConf(sparkConf) with Logging { override val resourceNamePrefix: String = { get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX).getOrElse( @@ -148,7 +149,8 @@ private[spark] class KubernetesExecutorConf( executorCustomLabels ++ presetLabels } - override def environment: Map[String, String] = sparkConf.getExecutorEnv.toMap + override def environment: Map[String, String] = sparkConf.getExecutorEnv.filter( + p => checkExecutorEnvKey(p._1)).toMap override def annotations: Map[String, String] = { KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) @@ -166,6 +168,20 @@ private[spark] class KubernetesExecutorConf( KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) } + private def checkExecutorEnvKey(key: String): Boolean = { + // Pattern for matching an executorEnv key, which meets certain naming rules. + val executorEnvRegex = "[-._a-zA-Z][-._a-zA-Z0-9]*".r + if (executorEnvRegex.pattern.matcher(key).matches()) { + true + } else { + logWarning(s"Invalid key: $key: " + + "a valid environment variable name must consist of alphabetic characters, " + + "digits, '_', '-', or '.', and must not start with a digit." + + s"Regex used for validation is '$executorEnvRegex')") + false + } + } + } private[spark] object KubernetesConf { 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 d6487556a371e..d88bd5858bc94 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 @@ -216,6 +216,9 @@ private[spark] class BasicExecutorFeatureStep( .endSpec() .build() + kubernetesConf.get(KUBERNETES_EXECUTOR_SCHEDULER_NAME) + .foreach(executorPod.getSpec.setSchedulerName) + SparkPod(executorPod, containerWithLimitCores) } } 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 11bbad9c480a1..8e5532d70b5da 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 @@ -86,15 +86,12 @@ private[spark] object ClientArguments { * @param builder Responsible for building the base driver pod based on a composition of * implemented features. * @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 watcher a watcher that monitors and logs the application status */ private[spark] class Client( conf: KubernetesDriverConf, builder: KubernetesDriverBuilder, kubernetesClient: KubernetesClient, - waitForAppCompletion: Boolean, watcher: LoggingPodStatusWatcher) extends Logging { def run(): Unit = { @@ -124,10 +121,11 @@ private[spark] class Client( .endVolume() .endSpec() .build() + val driverPodName = resolvedDriverPod.getMetadata.getName Utils.tryWithResource( kubernetesClient .pods() - .withName(resolvedDriverPod.getMetadata.getName) + .withName(driverPodName) .watch(watcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { @@ -141,16 +139,8 @@ 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 ${conf.appName} with submission ID ${sId} to finish...") - watcher.awaitCompletion() - logInfo(s"Application ${conf.appName} with submission ID ${sId} finished.") - } else { - logInfo(s"Deployed Spark application ${conf.appName} with " + - s"submission ID ${sId} into Kubernetes.") - } + val sId = Seq(conf.namespace, driverPodName).mkString(":") + watcher.watchOrStop(sId) } } @@ -199,13 +189,11 @@ private[spark] class KubernetesClientApplication extends SparkApplication { } private def run(clientArguments: ClientArguments, sparkConf: SparkConf): Unit = { - val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // For constructing the app ID, we can't use the Spark application name, as the app ID is going // to be added as a label to group resources belonging to the same application. Label values are // considerably restrictive, e.g. must be no longer than 63 characters in length. So we generate // 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 kubernetesConf = KubernetesConf.createDriverConf( sparkConf, kubernetesAppId, @@ -215,9 +203,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { // 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")) - val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None - - val watcher = new LoggingPodStatusWatcherImpl(kubernetesAppId, loggingInterval) + val watcher = new LoggingPodStatusWatcherImpl(kubernetesConf) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, @@ -231,7 +217,6 @@ private[spark] class KubernetesClientApplication extends SparkApplication { kubernetesConf, new KubernetesDriverBuilder(), kubernetesClient, - waitForAppCompletion, watcher) client.run() } 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 f16d1f3be7a6c..ce3c80c0f85b1 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 @@ -16,49 +16,36 @@ */ package org.apache.spark.deploy.k8s.submit -import java.util.concurrent.{CountDownLatch, TimeUnit} - 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.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.KubernetesDriverConf import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.internal.Logging -import org.apache.spark.util.ThreadUtils private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { - def awaitCompletion(): Unit + def watchOrStop(submissionId: String): Unit } /** * A monitor for the running Kubernetes pod of a Spark application. Status logging occurs on * every state change and also at an interval for liveness. * - * @param appId application ID. - * @param maybeLoggingInterval ms between each state request. If provided, must be a positive - * number. + * @param conf kubernetes driver conf. */ -private[k8s] class LoggingPodStatusWatcherImpl( - appId: String, - maybeLoggingInterval: Option[Long]) +private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) extends LoggingPodStatusWatcher with Logging { - private val podCompletedFuture = new CountDownLatch(1) - // start timer for periodic logging - private val scheduler = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("logging-pod-status-watcher") - private val logRunnable: Runnable = () => logShortStatus() + private val appId = conf.appId + + private var podCompleted = false private var pod = Option.empty[Pod] private def phase: String = pod.map(_.getStatus.getPhase).getOrElse("unknown") - def start(): Unit = { - maybeLoggingInterval.foreach { interval => - scheduler.scheduleAtFixedRate(logRunnable, 0, interval, TimeUnit.MILLISECONDS) - } - } - override def eventReceived(action: Action, pod: Pod): Unit = { this.pod = Option(pod) action match { @@ -78,11 +65,7 @@ private[k8s] class LoggingPodStatusWatcherImpl( closeWatch() } - private def logShortStatus() = { - logInfo(s"Application status for $appId (phase: $phase)") - } - - private def logLongStatus() = { + private def logLongStatus(): Unit = { logInfo("State changed, new state: " + pod.map(formatPodState).getOrElse("unknown")) } @@ -90,15 +73,25 @@ private[k8s] class LoggingPodStatusWatcherImpl( phase == "Succeeded" || phase == "Failed" } - private def closeWatch(): Unit = { - podCompletedFuture.countDown() - scheduler.shutdown() + private def closeWatch(): Unit = synchronized { + podCompleted = true + this.notifyAll() } - override def awaitCompletion(): Unit = { - podCompletedFuture.await() - logInfo(pod.map { p => - s"Container final statuses:\n\n${containersDescription(p)}" - }.getOrElse("No containers were found in the driver pod.")) + override def watchOrStop(sId: String): Unit = if (conf.get(WAIT_FOR_APP_COMPLETION)) { + logInfo(s"Waiting for application ${conf.appName} with submission ID $sId to finish...") + val interval = conf.get(REPORT_INTERVAL) + synchronized { + while (!podCompleted) { + wait(interval) + logInfo(s"Application status for $appId (phase: $phase)") + } + } + logInfo( + pod.map { p => s"Container final statuses:\n\n${containersDescription(p)}" } + .getOrElse("No containers were found in the driver pod.")) + logInfo(s"Application ${conf.appName} with submission ID $sId finished") + } else { + logInfo(s"Deployed Spark application ${conf.appName} with submission ID $sId into Kubernetes") } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala index d50ea85d3757b..c175308590964 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala @@ -22,7 +22,7 @@ private[spark] trait ExecutorPodsSnapshotsStore { def addSubscriber (processBatchIntervalMillis: Long) - (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit) + (onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit): Unit def stop(): Unit 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 d51b1e661bb1e..5591974c564e8 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 @@ -44,6 +44,12 @@ class KubernetesConfSuite extends SparkFunSuite { "customEnvKey2" -> "customEnvValue2") private val DRIVER_POD = new PodBuilder().build() private val EXECUTOR_ID = "executor-id" + private val EXECUTOR_ENV_VARS = Map( + "spark.executorEnv.1executorEnvVars1/var1" -> "executorEnvVars1", + "spark.executorEnv.executorEnvVars2*var2" -> "executorEnvVars2", + "spark.executorEnv.executorEnvVars3_var3" -> "executorEnvVars3", + "spark.executorEnv.executorEnvVars4-var4" -> "executorEnvVars4", + "spark.executorEnv.executorEnvVars5-var5" -> "executorEnvVars5/var5") test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") { val sparkConf = new SparkConf(false) @@ -132,4 +138,22 @@ class KubernetesConfSuite extends SparkFunSuite { assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) } + + test("Verify that executorEnv key conforms to the regular specification") { + val sparkConf = new SparkConf(false) + EXECUTOR_ENV_VARS.foreach { case (key, value) => + sparkConf.set(key, value) + } + + val conf = KubernetesConf.createExecutorConf( + sparkConf, + EXECUTOR_ID, + KubernetesTestConf.APP_ID, + Some(DRIVER_POD)) + assert(conf.environment === + Map( + "executorEnvVars3_var3" -> "executorEnvVars3", + "executorEnvVars4-var4" -> "executorEnvVars4", + "executorEnvVars5-var5" -> "executorEnvVars5/var5")) + } } 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 aa421be6e8412..5d49ac0bbaafa 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 @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} import org.mockito.Mockito.{verify, when} import org.scalatest.BeforeAndAfter -import org.scalatest.mockito.MockitoSugar._ +import org.scalatestplus.mockito.MockitoSugar._ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s._ @@ -146,7 +146,6 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { kconf, driverBuilder, kubernetesClient, - false, loggingPodStatusWatcher) submissionClient.run() verify(podOperations).create(FULL_EXPECTED_POD) @@ -157,7 +156,6 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { kconf, driverBuilder, kubernetesClient, - false, loggingPodStatusWatcher) submissionClient.run() val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues @@ -181,9 +179,8 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { kconf, driverBuilder, kubernetesClient, - true, loggingPodStatusWatcher) submissionClient.run() - verify(loggingPodStatusWatcher).awaitCompletion() + verify(loggingPodStatusWatcher).watchOrStop(kconf.namespace + ":driver") } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 871d34b11e174..b6eeff1cd18a9 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -15,7 +15,7 @@ # limitations under the License. # -FROM openjdk:8-alpine +FROM openjdk:8-jdk-slim ARG spark_uid=185 @@ -27,9 +27,9 @@ ARG spark_uid=185 # docker build -t spark:latest -f kubernetes/dockerfiles/spark/Dockerfile . RUN set -ex && \ - apk upgrade --no-cache && \ + apt-get update && \ ln -s /lib /lib64 && \ - apk add --no-cache bash tini libc6-compat linux-pam krb5 krb5-libs nss && \ + apt install -y bash tini libc6 libpam-modules krb5-user libnss3 && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/examples && \ mkdir -p /opt/spark/work-dir && \ @@ -37,7 +37,8 @@ RUN set -ex && \ rm /bin/sh && \ ln -sv /bin/bash /bin/sh && \ echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && \ - chgrp root /etc/passwd && chmod ug+rw /etc/passwd + chgrp root /etc/passwd && chmod ug+rw /etc/passwd && \ + rm -rf /var/cache/apt/* COPY jars /opt/spark/jars COPY bin /opt/spark/bin 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 34d449c9f08b9..c65824c3f9a88 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 @@ -25,7 +25,7 @@ USER 0 RUN mkdir ${SPARK_HOME}/R -RUN apk add --no-cache R R-dev +RUN apt install -y r-base r-base-dev && rm -rf /var/cache/apt/* COPY R ${SPARK_HOME}/R ENV R_HOME /usr/lib/R 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 8237c9223223a..a7dce048710d8 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 @@ -25,17 +25,15 @@ USER 0 RUN mkdir ${SPARK_HOME}/python # TODO: Investigate running both pip and pip3 via virtualenvs -RUN apk add --no-cache python && \ - apk add --no-cache python3 && \ - python -m ensurepip && \ - python3 -m ensurepip && \ +RUN apt install -y python python-pip && \ + apt install -y python3 python3-pip && \ # We remove ensurepip since it adds no functionality since pip is # installed on the image and it just takes up 1.6MB on the image rm -r /usr/lib/python*/ensurepip && \ pip install --upgrade pip setuptools && \ # You may install with python3 packages by using pip3.6 # Removed the .cache to save space - rm -r /root/.cache + rm -r /root/.cache && rm -rf /var/cache/apt/* COPY python/pyspark ${SPARK_HOME}/python/pyspark COPY python/lib ${SPARK_HOME}/python/lib 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 2097fb8865de9..4fe8df61ef569 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -94,4 +94,4 @@ case "$1" in esac # Execute the container CMD under tini for better hygiene -exec /sbin/tini -s -- "${CMD[@]}" +exec /usr/bin/tini -s -- "${CMD[@]}" diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 84d99115a93ac..c79caff164ce8 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -29,7 +29,7 @@ 1.3.0 1.4.0 - 4.4.2 + 4.6.1 3.2.2 1.0 kubernetes-integration-tests 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 072730e97e778..9e04b963fc40e 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 @@ -85,8 +85,8 @@ then # 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" + # Build SparkR image + LANGUAGE_BINDING_BUILD_ARGS="$LANGUAGE_BINDING_BUILD_ARGS -R $DOCKER_FILE_BASE_PATH/bindings/R/Dockerfile" # Unset SPARK_HOME to let the docker-image-tool script detect SPARK_HOME. Otherwise, it cannot # indicate the unpacked directory as its home. See SPARK-28550. 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 80d5f239a09cc..0d4fcccc35cf9 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 @@ -42,7 +42,7 @@ import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite with Logging with Eventually with Matchers { + with DepsTestsSuite with RTestsSuite with Logging with Eventually with Matchers { import KubernetesSuite._ 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 bc1247ad78936..c6139dd268d5c 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 @@ -106,7 +106,7 @@ private[mesos] object MesosClusterDispatcher extends Logging with CommandLineUtils { - override def main(args: Array[String]) { + override def main(args: Array[String]): Unit = { Thread.setDefaultUncaughtExceptionHandler(new SparkUncaughtExceptionHandler) Utils.initDaemon(log) val conf = new SparkConf diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala index 15bbe60d6c8fb..c0cdcda14291f 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -37,7 +37,7 @@ private[spark] class MesosClusterUI( def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort - override def initialize() { + override def initialize(): Unit = { attachPage(new MesosClusterPage(this)) attachPage(new DriverPage(this)) addStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR) 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 bfa88d68d06c2..213d33cb4f0f2 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 @@ -43,7 +43,7 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null - override def statusUpdate(taskId: Long, state: TaskState.TaskState, data: ByteBuffer) { + override def statusUpdate(taskId: Long, state: TaskState.TaskState, data: ByteBuffer): Unit = { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() driver.sendStatusUpdate(MesosTaskStatus.newBuilder() .setTaskId(mesosTaskId) @@ -56,7 +56,7 @@ private[spark] class MesosExecutorBackend driver: ExecutorDriver, executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, - slaveInfo: SlaveInfo) { + slaveInfo: SlaveInfo): Unit = { // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend. val cpusPerTask = executorInfo.getResourcesList.asScala @@ -85,7 +85,7 @@ private[spark] class MesosExecutorBackend env) } - override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { + override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo): Unit = { val taskDescription = TaskDescription.decode(taskInfo.getData.asReadOnlyByteBuffer()) if (executor == null) { logError("Received launchTask but executor was null") @@ -96,11 +96,11 @@ private[spark] class MesosExecutorBackend } } - override def error(d: ExecutorDriver, message: String) { + override def error(d: ExecutorDriver, message: String): Unit = { logError("Error from Mesos: " + message) } - override def killTask(d: ExecutorDriver, t: TaskID) { + override def killTask(d: ExecutorDriver, t: TaskID): Unit = { if (executor == null) { logError("Received KillTask but executor was null") } else { @@ -110,20 +110,20 @@ private[spark] class MesosExecutorBackend } } - override def reregistered(d: ExecutorDriver, p2: SlaveInfo) {} + override def reregistered(d: ExecutorDriver, p2: SlaveInfo): Unit = {} - override def disconnected(d: ExecutorDriver) {} + override def disconnected(d: ExecutorDriver): Unit = {} - override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} + override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]): Unit = {} - override def shutdown(d: ExecutorDriver) {} + override def shutdown(d: ExecutorDriver): Unit = {} } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend extends Logging { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { Utils.initDaemon(log) // Create a new Executor and start it running val runner = new MesosExecutorBackend() 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 a54bca800a007..e916125ffdb67 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 @@ -187,7 +187,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( id.toString } - override def start() { + override def start(): Unit = { super.start() if (sc.deployMode == "client") { @@ -322,12 +322,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } } - override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID): Unit = {} override def registered( driver: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, - masterInfo: MasterInfo) { + masterInfo: MasterInfo): Unit = { this.appId = frameworkId.getValue this.mesosExternalShuffleClient.foreach(_.init(appId)) @@ -341,11 +341,11 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( totalCoreCount.get >= maxCoresOption.getOrElse(0) * minRegisteredRatio } - override def disconnected(d: org.apache.mesos.SchedulerDriver) { + override def disconnected(d: org.apache.mesos.SchedulerDriver): Unit = { launcherBackend.setState(SparkAppHandle.State.SUBMITTED) } - override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) { + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo): Unit = { launcherBackend.setState(SparkAppHandle.State.RUNNING) } @@ -353,7 +353,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( * Method called by Mesos to offer resources on slaves. We respond by launching an executor, * unless we've already launched more than we wanted to. */ - override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]): Unit = { stateLock.synchronized { if (stopCalled) { logDebug("Ignoring offers during shutdown") @@ -613,7 +613,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( return true } - override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus): Unit = { val taskId = status.getTaskId.getValue val slaveId = status.getSlaveId.getValue val state = mesosToTaskState(status.getState) @@ -675,12 +675,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } } - override def error(d: org.apache.mesos.SchedulerDriver, message: String) { + override def error(d: org.apache.mesos.SchedulerDriver, message: String): Unit = { logError(s"Mesos error: $message") scheduler.error(message) } - override def stop() { + override def stop(): Unit = { stopSchedulerBackend() launcherBackend.setState(SparkAppHandle.State.FINISHED) launcherBackend.close() @@ -722,7 +722,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]): Unit = {} /** * Called when a slave is lost or a Mesos task finished. Updates local view on 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 a03fecdb2abee..e2a99148dd799 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 @@ -74,7 +74,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( @volatile var appId: String = _ - override def start() { + override def start(): Unit = { classLoader = Thread.currentThread.getContextClassLoader val driver = createSchedulerDriver( master, @@ -184,12 +184,12 @@ private[spark] class MesosFineGrainedSchedulerBackend( execArgs } - override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID): Unit = {} override def registered( driver: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, - masterInfo: MasterInfo) { + masterInfo: MasterInfo): Unit = { inClassLoader() { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) @@ -208,9 +208,9 @@ private[spark] class MesosFineGrainedSchedulerBackend( } } - override def disconnected(d: org.apache.mesos.SchedulerDriver) {} + override def disconnected(d: org.apache.mesos.SchedulerDriver): Unit = {} - override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo): Unit = {} private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = { val builder = new StringBuilder @@ -229,7 +229,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ - override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]): Unit = { inClassLoader() { // Fail first on offers with unmet constraints val (offersMatchingConstraints, offersNotMatchingConstraints) = @@ -368,7 +368,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( (taskInfo, finalResources.asJava) } - override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus): Unit = { inClassLoader() { val tid = status.getTaskId.getValue.toLong val state = mesosToTaskState(status.getState) @@ -386,7 +386,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( } } - override def error(d: org.apache.mesos.SchedulerDriver, message: String) { + override def error(d: org.apache.mesos.SchedulerDriver, message: String): Unit = { inClassLoader() { logError("Mesos error: " + message) markErr() @@ -394,18 +394,18 @@ private[spark] class MesosFineGrainedSchedulerBackend( } } - override def stop() { + override def stop(): Unit = { if (schedulerDriver != null) { schedulerDriver.stop() } } - override def reviveOffers() { + override def reviveOffers(): Unit = { schedulerDriver.reviveOffers() } override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]): Unit = {} /** * Remove executor associated with slaveId in a thread safe manner. @@ -418,7 +418,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( } private def recordSlaveLost( - d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { + d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason): Unit = { inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) removeExecutor(slaveId.getValue, reason.toString) @@ -426,12 +426,15 @@ private[spark] class MesosFineGrainedSchedulerBackend( } } - override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID) { + override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { recordSlaveLost(d, slaveId, SlaveLost()) } override def executorLost( - d: org.apache.mesos.SchedulerDriver, executorId: ExecutorID, slaveId: SlaveID, status: Int) { + d: org.apache.mesos.SchedulerDriver, + executorId: ExecutorID, + slaveId: SlaveID, + status: Int): Unit = { logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, slaveId.getValue)) recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true)) 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 f60b3b8db194c..ed3bd358d4082 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 @@ -149,7 +149,7 @@ trait MesosSchedulerUtils extends Logging { // until the scheduler exists new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { setDaemon(true) - override def run() { + override def run(): Unit = { try { val ret = newDriver.run() logInfo("driver.run() returned with code " + ret) @@ -552,7 +552,7 @@ trait MesosSchedulerUtils extends Logging { * the same frameworkID. To enforce that only the first driver registers with the configured * framework ID, the driver calls this method after the first registration. */ - def unsetFrameworkID(sc: SparkContext) { + def unsetFrameworkID(sc: SparkContext): Unit = { sc.conf.remove(mesosConfig.DRIVER_FRAMEWORK_ID) System.clearProperty(mesosConfig.DRIVER_FRAMEWORK_ID.key) } 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 index 1f83149a05652..344fc38c84fb1 100644 --- 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 @@ -19,7 +19,7 @@ package org.apache.spark.deploy.rest.mesos import javax.servlet.http.HttpServletResponse -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.TestPrematureExit @@ -45,7 +45,7 @@ class MesosRestServerSuite extends SparkFunSuite testOverheadMemory(conf, "2000M", 3000) } - def testOverheadMemory(conf: SparkConf, driverMemory: String, expectedResult: Int) { + def testOverheadMemory(conf: SparkConf, driverMemory: String, expectedResult: Int): Unit = { conf.set("spark.master", "testmaster") conf.set("spark.app.name", "testapp") conf.set(config.DRIVER_MEMORY.key, driverMemory) 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 e8520061ac38d..7f409ae224fc3 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 @@ -22,7 +22,7 @@ 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) { + def testURL(masterURL: String, expectedClass: Class[_], coarse: Boolean): Unit = { 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 924a991c50f23..9a50142b51d97 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 @@ -27,7 +27,7 @@ import org.apache.mesos.SchedulerDriver import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito._ -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} import org.apache.spark.deploy.Command 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 f810da17e6c44..8f6ae5904f394 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 @@ -28,7 +28,7 @@ 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.scalatestplus.mockito.MockitoSugar import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.deploy.mesos.{config => mesosConfig} @@ -809,7 +809,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } } - private def setBackend(sparkConfVars: Map[String, String] = null, home: String = "/path") { + private def setBackend(sparkConfVars: Map[String, String] = null, + home: String = "/path"): Unit = { initializeSparkConf(sparkConfVars, home) sc = new SparkContext(sparkConf) 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 79a57ad031b71..a5bd34888a0a6 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 @@ -34,7 +34,7 @@ import org.apache.mesos.Protos.Value.Scalar import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} import org.mockito.Mockito._ -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} 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 8be24cdbc949a..0ed6fe66c56eb 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 @@ -26,7 +26,7 @@ import com.google.common.io.Files import org.apache.mesos.Protos.{FrameworkInfo, Resource, Value} import org.mockito.Mockito._ import org.scalatest._ -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} import org.apache.spark.deploy.mesos.{config => mesosConfig} @@ -63,12 +63,12 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS def arePortsEqual(array1: Array[(Long, Long)], array2: Array[(Long, Long)]) : Boolean = { - array1.sortBy(identity).deep == array2.sortBy(identity).deep + array1.sortBy(identity).sameElements(array2.sortBy(identity)) } def arePortsEqual(array1: Array[Long], array2: Array[Long]) : Boolean = { - array1.sortBy(identity).deep == array2.sortBy(identity).deep + array1.sortBy(identity).sameElements(array2.sortBy(identity)) } def getRangesFromResources(resources: List[Resource]): List[(Long, Long)] = { 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 68035e4321e01..2e9576e3355d1 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 @@ -716,7 +716,7 @@ private[spark] class ApplicationMaster( .getMethod("main", classOf[Array[String]]) val userThread = new Thread { - override def run() { + override def run(): Unit = { try { if (!Modifier.isStatic(mainMethod.getModifiers)) { logError(s"Could not find static main method in object ${args.userClass}") 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 c10206c847271..d2275980814e3 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 @@ -82,7 +82,7 @@ class ApplicationMasterArguments(val args: Array[String]) { userArgs = userArgsBuffer.toList } - def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { // scalastyle:off println if (unknownParam != null) { System.err.println("Unknown/unsupported param " + unknownParam) 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 c7c495fef853f..a647bb4f3d68c 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 @@ -42,7 +42,7 @@ private[spark] class YarnClientSchedulerBackend( * Create a Yarn client to submit an application to the ResourceManager. * This waits until the application is running. */ - override def start() { + override def start(): Unit = { super.start() val driverHost = conf.get(config.DRIVER_HOST_ADDRESS) @@ -109,7 +109,7 @@ private[spark] class YarnClientSchedulerBackend( private class MonitorThread extends Thread { private var allowInterrupt = true - override def run() { + override def run(): Unit = { try { val YarnAppReport(_, state, diags) = client.monitorApplication(appId.get, logApplicationReport = false) @@ -148,7 +148,7 @@ private[spark] class YarnClientSchedulerBackend( /** * Stop the scheduler. This assumes `start()` has already been called. */ - override def stop() { + override def stop(): Unit = { assert(client != null, "Attempted to stop this scheduler before starting it!") if (monitorThread != null) { monitorThread.stopMonitor() 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 1f622a02a62ae..1812a55c39589 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 @@ -28,7 +28,7 @@ private[spark] class YarnClusterScheduler(sc: SparkContext) extends YarnSchedule logInfo("Created YarnClusterScheduler") - override def postStartHook() { + override def postStartHook(): Unit = { ApplicationMaster.sparkContextInitialized(sc) super.postStartHook() logInfo("YarnClusterScheduler.postStartHook done") 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 b5575a10a05a0..e70a78d3c4c8d 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 @@ -27,7 +27,7 @@ private[spark] class YarnClusterSchedulerBackend( sc: SparkContext) extends YarnSchedulerBackend(scheduler, sc) { - override def start() { + override def start(): Unit = { val attemptId = ApplicationMaster.getAttemptId bindToYarn(attemptId.getApplicationId(), Some(attemptId)) super.start() 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 e16857e84887c..f8ef0d08d829c 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 @@ -63,7 +63,7 @@ abstract class BaseYarnClusterSuite def newYarnConfig(): YarnConfiguration - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() tempDir = Utils.createTempDir() @@ -115,7 +115,7 @@ abstract class BaseYarnClusterSuite File.createTempFile("token", ".txt", hadoopConfDir) } - override def afterAll() { + override def afterAll(): Unit = { try { yarnCluster.stop() } finally { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala index b091fec926c4c..c04b4e5cb559e 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType import org.apache.hadoop.yarn.api.records.LocalResourceVisibility import org.apache.hadoop.yarn.util.ConverterUtils import org.mockito.Mockito.when -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.yarn.config._ 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 847fc3773de59..bcf16f6dab111 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 @@ -523,7 +523,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val mapAppConf = mapYARNAppConf ++ mapMRAppConf } - def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any) { + def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any): Unit = { val conf = new Configuration m.foreach { case (k, v) => conf.set(k, v, "ClientSpec") } testCode(conf) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala index afb4b691b52de..29f1c0512fbd5 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala @@ -30,11 +30,11 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B def createContainerRequest(nodes: Array[String]): ContainerRequest = new ContainerRequest(containerResource, nodes, null, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - override def beforeEach() { + override def beforeEach(): Unit = { yarnAllocatorSuite.beforeEach() } - override def afterEach() { + override def afterEach(): Unit = { yarnAllocatorSuite.afterEach() } 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 6f47a418f9180..6216d473882e6 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 @@ -69,7 +69,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter var containerNum = 0 - override def beforeEach() { + override def beforeEach(): Unit = { super.beforeEach() rmClient = AMRMClient.createAMRMClient() rmClient.init(conf) @@ -77,7 +77,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock = new ManualClock() } - override def afterEach() { + override def afterEach(): Unit = { try { rmClient.stop() } finally { 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 7264e2e51ee45..b7c9e83446012 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 @@ -333,7 +333,7 @@ private[spark] class SaveExecutorInfo extends SparkListener { var driverLogs: Option[collection.Map[String, String]] = None var driverAttributes: Option[collection.Map[String, String]] = None - override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + override def onExecutorAdded(executor: SparkListenerExecutorAdded): Unit = { addedExecutorInfos(executor.executorId) = executor.executorInfo } 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 835f0736c5a1e..a87820b1528ad 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 @@ -21,7 +21,7 @@ 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.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.scheduler.TaskSchedulerImpl diff --git a/sql/catalyst/benchmarks/HashBenchmark-jdk11-results.txt b/sql/catalyst/benchmarks/HashBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..d6452891a3ddd --- /dev/null +++ b/sql/catalyst/benchmarks/HashBenchmark-jdk11-results.txt @@ -0,0 +1,70 @@ +================================================================================================ +single ints +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For single ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 3285 3286 1 163.4 6.1 1.0X +codegen version 6838 6838 0 78.5 12.7 0.5X +codegen version 64-bit 6247 6247 0 85.9 11.6 0.5X +codegen HiveHash version 4927 4927 0 109.0 9.2 0.7X + + +================================================================================================ +single longs +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For single longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 3615 3615 0 148.5 6.7 1.0X +codegen version 9630 9633 5 55.8 17.9 0.4X +codegen version 64-bit 6763 6764 2 79.4 12.6 0.5X +codegen HiveHash version 5709 5711 2 94.0 10.6 0.6X + + +================================================================================================ +normal +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For normal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 2946 2965 27 0.7 1404.7 1.0X +codegen version 2386 2386 1 0.9 1137.7 1.2X +codegen version 64-bit 895 895 0 2.3 426.8 3.3X +codegen HiveHash version 4454 4463 14 0.5 2123.6 0.7X + + +================================================================================================ +array +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 2325 2374 68 0.1 17740.6 1.0X +codegen version 4928 4977 69 0.0 37597.1 0.5X +codegen version 64-bit 3404 3408 5 0.0 25973.0 0.7X +codegen HiveHash version 1491 1512 30 0.1 11376.3 1.6X + + +================================================================================================ +map +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 0 0 0 45.1 22.2 1.0X +codegen version 271 278 8 0.0 66062.7 0.0X +codegen version 64-bit 208 210 2 0.0 50775.0 0.0X +codegen HiveHash version 52 53 0 0.1 12794.9 0.0X + + diff --git a/sql/catalyst/benchmarks/HashBenchmark-results.txt b/sql/catalyst/benchmarks/HashBenchmark-results.txt index 2459b35c75bb5..8075df73399b9 100644 --- a/sql/catalyst/benchmarks/HashBenchmark-results.txt +++ b/sql/catalyst/benchmarks/HashBenchmark-results.txt @@ -2,69 +2,69 @@ single ints ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For single ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 5615 / 5616 95.6 10.5 1.0X -codegen version 8400 / 8407 63.9 15.6 0.7X -codegen version 64-bit 8139 / 8145 66.0 15.2 0.7X -codegen HiveHash version 7213 / 7348 74.4 13.4 0.8X +Hash For single ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 3364 3364 0 159.6 6.3 1.0X +codegen version 6921 6924 4 77.6 12.9 0.5X +codegen version 64-bit 6139 6140 2 87.5 11.4 0.5X +codegen HiveHash version 5097 5099 2 105.3 9.5 0.7X ================================================================================================ single longs ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For single longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 6053 / 6054 88.7 11.3 1.0X -codegen version 9367 / 9369 57.3 17.4 0.6X -codegen version 64-bit 8041 / 8051 66.8 15.0 0.8X -codegen HiveHash version 7546 / 7575 71.1 14.1 0.8X +Hash For single longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 3838 3838 0 139.9 7.1 1.0X +codegen version 9690 9703 19 55.4 18.0 0.4X +codegen version 64-bit 6876 6877 1 78.1 12.8 0.6X +codegen HiveHash version 5717 5718 1 93.9 10.6 0.7X ================================================================================================ normal ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For normal: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 3181 / 3182 0.7 1517.0 1.0X -codegen version 2403 / 2403 0.9 1145.7 1.3X -codegen version 64-bit 915 / 916 2.3 436.2 3.5X -codegen HiveHash version 4505 / 4527 0.5 2148.3 0.7X +Hash For normal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 2914 2918 6 0.7 1389.6 1.0X +codegen version 2337 2341 6 0.9 1114.5 1.2X +codegen version 64-bit 910 911 3 2.3 433.9 3.2X +codegen HiveHash version 4479 4480 1 0.5 2135.9 0.7X ================================================================================================ array ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 1828 / 1844 0.1 13946.1 1.0X -codegen version 3678 / 3804 0.0 28058.2 0.5X -codegen version 64-bit 2925 / 2931 0.0 22317.8 0.6X -codegen HiveHash version 1216 / 1217 0.1 9280.0 1.5X +Hash For array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 2081 2133 74 0.1 15878.7 1.0X +codegen version 4385 4506 172 0.0 33452.2 0.5X +codegen version 64-bit 3518 3552 48 0.0 26842.8 0.6X +codegen HiveHash version 1715 1752 52 0.1 13085.4 1.2X ================================================================================================ map ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 0 / 0 44.3 22.6 1.0X -codegen version 176 / 176 0.0 42978.8 0.0X -codegen version 64-bit 173 / 175 0.0 42214.3 0.0X -codegen HiveHash version 44 / 44 0.1 10659.9 0.0X +Hash For map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +interpreted version 0 0 0 50.4 19.8 1.0X +codegen version 195 204 8 0.0 47681.7 0.0X +codegen version 64-bit 186 190 3 0.0 45296.8 0.0X +codegen HiveHash version 48 48 0 0.1 11610.3 0.0X diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk11-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..d8d43068929ed --- /dev/null +++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk11-results.txt @@ -0,0 +1,77 @@ +================================================================================================ +Benchmark for MurMurHash 3 and xxHash64 +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 8: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 14 14 0 147.0 6.8 1.0X +xxHash 64-bit 18 18 0 119.7 8.4 0.8X +HiveHasher 16 16 0 129.9 7.7 0.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 16: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 23 23 0 89.8 11.1 1.0X +xxHash 64-bit 22 23 0 93.3 10.7 1.0X +HiveHasher 26 26 0 79.4 12.6 0.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 24: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 32 32 0 65.5 15.3 1.0X +xxHash 64-bit 26 26 0 80.8 12.4 1.2X +HiveHasher 41 41 0 50.8 19.7 0.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 31: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 49 49 0 43.2 23.2 1.0X +xxHash 64-bit 44 44 0 48.0 20.8 1.1X +HiveHasher 55 56 0 37.8 26.4 0.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 95: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 127 127 0 16.5 60.5 1.0X +xxHash 64-bit 83 83 0 25.3 39.5 1.5X +HiveHasher 196 196 0 10.7 93.3 0.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 287: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 327 327 0 6.4 155.9 1.0X +xxHash 64-bit 138 138 0 15.2 65.9 2.4X +HiveHasher 628 628 0 3.3 299.6 0.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 1055: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 1145 1145 0 1.8 545.9 1.0X +xxHash 64-bit 370 371 0 5.7 176.6 3.1X +HiveHasher 2325 2325 0 0.9 1108.6 0.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 2079: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 2209 2211 3 0.9 1053.4 1.0X +xxHash 64-bit 615 615 0 3.4 293.0 3.6X +HiveHasher 4590 4590 0 0.5 2188.7 0.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 8223: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 8687 8702 22 0.2 4142.2 1.0X +xxHash 64-bit 2033 2034 1 1.0 969.5 4.3X +HiveHasher 18216 18218 2 0.1 8686.1 0.5X + + diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt index a4304ee3b5f60..83bd970e14392 100644 --- a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt +++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt @@ -2,76 +2,76 @@ Benchmark for MurMurHash 3 and xxHash64 ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 8: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 16 / 16 127.7 7.8 1.0X -xxHash 64-bit 23 / 23 90.7 11.0 0.7X -HiveHasher 16 / 16 134.8 7.4 1.1X +Hash byte arrays with length 8: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 15 15 0 138.0 7.2 1.0X +xxHash 64-bit 17 17 0 125.7 8.0 0.9X +HiveHasher 16 16 0 134.4 7.4 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 16: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 26 / 26 79.5 12.6 1.0X -xxHash 64-bit 26 / 27 79.3 12.6 1.0X -HiveHasher 30 / 30 70.1 14.3 0.9X +Hash byte arrays with length 16: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 24 24 0 86.8 11.5 1.0X +xxHash 64-bit 22 22 0 96.5 10.4 1.1X +HiveHasher 31 31 0 66.8 15.0 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 24: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 36 / 36 58.1 17.2 1.0X -xxHash 64-bit 30 / 30 70.2 14.2 1.2X -HiveHasher 45 / 45 46.4 21.5 0.8X +Hash byte arrays with length 24: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 34 34 0 61.9 16.2 1.0X +xxHash 64-bit 26 26 0 79.6 12.6 1.3X +HiveHasher 48 48 0 44.0 22.7 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 31: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 50 / 50 41.8 23.9 1.0X -xxHash 64-bit 43 / 43 49.3 20.3 1.2X -HiveHasher 58 / 58 35.9 27.8 0.9X +Hash byte arrays with length 31: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 50 50 0 41.9 23.9 1.0X +xxHash 64-bit 40 40 0 52.4 19.1 1.3X +HiveHasher 61 61 0 34.4 29.1 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 95: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 132 / 132 15.9 62.7 1.0X -xxHash 64-bit 79 / 79 26.7 37.5 1.7X -HiveHasher 198 / 199 10.6 94.6 0.7X +Hash byte arrays with length 95: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 132 133 0 15.9 63.1 1.0X +xxHash 64-bit 77 78 0 27.4 36.5 1.7X +HiveHasher 209 209 0 10.0 99.6 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 287: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 334 / 334 6.3 159.3 1.0X -xxHash 64-bit 126 / 126 16.7 59.9 2.7X -HiveHasher 633 / 634 3.3 302.0 0.5X +Hash byte arrays with length 287: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 333 334 0 6.3 158.9 1.0X +xxHash 64-bit 123 123 0 17.1 58.6 2.7X +HiveHasher 630 630 0 3.3 300.3 0.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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 1055: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 1149 / 1149 1.8 547.9 1.0X -xxHash 64-bit 327 / 327 6.4 155.9 3.5X -HiveHasher 2338 / 2346 0.9 1114.6 0.5X +Hash byte arrays with length 1055: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 1151 1151 0 1.8 548.9 1.0X +xxHash 64-bit 321 321 0 6.5 153.2 3.6X +HiveHasher 2332 2332 1 0.9 1111.8 0.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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 2079: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 2215 / 2216 0.9 1056.1 1.0X -xxHash 64-bit 554 / 554 3.8 264.0 4.0X -HiveHasher 4609 / 4609 0.5 2197.5 0.5X +Hash byte arrays with length 2079: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 2213 2213 0 0.9 1055.2 1.0X +xxHash 64-bit 550 550 0 3.8 262.3 4.0X +HiveHasher 4599 4599 1 0.5 2192.8 0.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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 8223: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 8633 / 8643 0.2 4116.3 1.0X -xxHash 64-bit 1891 / 1892 1.1 901.6 4.6X -HiveHasher 18206 / 18206 0.1 8681.3 0.5X +Hash byte arrays with length 8223: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Murmur3_x86_32 8660 8664 6 0.2 4129.3 1.0X +xxHash 64-bit 1889 1893 6 1.1 900.6 4.6X +HiveHasher 18269 18272 5 0.1 8711.3 0.5X diff --git a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk11-results.txt b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..8b56fcbebc79c --- /dev/null +++ b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk11-results.txt @@ -0,0 +1,14 @@ +================================================================================================ +unsafe projection +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +unsafe projection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +single long 2218 2219 1 121.0 8.3 1.0X +single nullable long 3200 3201 1 83.9 11.9 0.7X +7 primitive types 7809 7813 5 34.4 29.1 0.3X +7 nullable primitive types 11906 11908 3 22.5 44.4 0.2X + + diff --git a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt index 43156dc6fc67f..7b30e2075f010 100644 --- a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt +++ b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt @@ -2,13 +2,13 @@ unsafe projection ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -unsafe projection: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -single long 2867 / 2868 93.6 10.7 1.0X -single nullable long 3915 / 3949 68.6 14.6 0.7X -7 primitive types 8166 / 8167 32.9 30.4 0.4X -7 nullable primitive types 12767 / 12767 21.0 47.6 0.2X +unsafe projection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +single long 2173 2173 0 123.5 8.1 1.0X +single nullable long 3156 3157 2 85.0 11.8 0.7X +7 primitive types 6725 6726 2 39.9 25.1 0.3X +7 nullable primitive types 11399 11400 2 23.5 42.5 0.2X diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 79d9b4951b151..bcebb225dfaca 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -111,7 +111,6 @@ com.univocity univocity-parsers - 2.7.3 jar @@ -170,4 +169,16 @@ + + + + scala-2.13 + + + org.scala-lang.modules + scala-parallel-collections_${scala.binary.version} + + + + 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 20be8e539cf58..1e89507411adb 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 @@ -79,20 +79,25 @@ singleTableSchema : colTypeList EOF ; +singleInterval + : INTERVAL? (intervalValue intervalUnit)+ EOF + ; + statement : query #statementDefault | ctes? dmlStatementNoWith #dmlStatement - | USE db=errorCapturingIdentifier #use - | CREATE database (IF NOT EXISTS)? db=errorCapturingIdentifier + | USE NAMESPACE? multipartIdentifier #use + | CREATE (database | NAMESPACE) (IF NOT EXISTS)? multipartIdentifier ((COMMENT comment=STRING) | locationSpec | - (WITH DBPROPERTIES tablePropertyList))* #createDatabase + (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace | ALTER database db=errorCapturingIdentifier SET DBPROPERTIES tablePropertyList #setDatabaseProperties + | ALTER database db=errorCapturingIdentifier + SET locationSpec #setDatabaseLocation | DROP database (IF EXISTS)? db=errorCapturingIdentifier (RESTRICT | CASCADE)? #dropDatabase - | SHOW DATABASES (LIKE? pattern=STRING)? #showDatabases - | SHOW NAMESPACES ((FROM | IN) multipartIdentifier)? + | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? (LIKE? pattern=STRING)? #showNamespaces | createTableHeader ('(' colTypeList ')')? tableProvider ((OPTIONS options=tablePropertyList) | @@ -123,7 +128,7 @@ statement (COMMENT comment=STRING) | (TBLPROPERTIES tableProps=tablePropertyList))* (AS? query)? #replaceTable - | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS + | ANALYZE TABLE multipartIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze | ALTER TABLE multipartIdentifier ADD (COLUMN | COLUMNS) @@ -193,30 +198,31 @@ statement ('(' key=tablePropertyKey ')')? #showTblProperties | SHOW COLUMNS (FROM | IN) tableIdentifier ((FROM | IN) db=errorCapturingIdentifier)? #showColumns - | SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions + | SHOW PARTITIONS multipartIdentifier 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? db=errorCapturingIdentifier #describeDatabase | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? - multipartIdentifier partitionSpec? describeColName? #describeTable + multipartIdentifier partitionSpec? describeColName? #describeTable | (DESC | DESCRIBE) QUERY? query #describeQuery - | REFRESH TABLE tableIdentifier #refreshTable + | REFRESH TABLE multipartIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource - | CACHE LAZY? TABLE tableIdentifier + | CACHE LAZY? TABLE multipartIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable - | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable + | UNCACHE TABLE (IF EXISTS)? multipartIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData - | TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable - | MSCK REPAIR TABLE tableIdentifier #repairTable + | TRUNCATE TABLE multipartIdentifier partitionSpec? #truncateTable + | MSCK REPAIR TABLE multipartIdentifier #repairTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable + | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -476,6 +482,14 @@ selectClause : SELECT (hints+=hint)* setQuantifier? namedExpressionSeq ; +setClause + : SET assign (',' assign)* + ; + +assign + : key=multipartIdentifier EQ value=expression + ; + whereClause : WHERE booleanExpression ; @@ -899,7 +913,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.parser.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.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. @@ -1008,6 +1022,7 @@ ansiNonReserved | MINUTES | MONTHS | MSCK + | NAMESPACE | NAMESPACES | NO | NULLS @@ -1028,6 +1043,7 @@ ansiNonReserved | POSITION | PRECEDING | PRINCIPALS + | PROPERTIES | PURGE | QUERY | RANGE @@ -1085,6 +1101,7 @@ ansiNonReserved | UNCACHE | UNLOCK | UNSET + | UPDATE | USE | VALUES | VIEW @@ -1094,9 +1111,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.parser.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.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`. +// Same definition as the one when `spark.sql.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. @@ -1258,6 +1275,7 @@ nonReserved | MONTH | MONTHS | MSCK + | NAMESPACE | NAMESPACES | NO | NOT @@ -1286,6 +1304,7 @@ nonReserved | PRECEDING | PRIMARY | PRINCIPALS + | PROPERTIES | PURGE | QUERY | RANGE @@ -1355,6 +1374,7 @@ nonReserved | UNKNOWN | UNLOCK | UNSET + | UPDATE | USE | USER | VALUES @@ -1519,6 +1539,7 @@ MINUTES: 'MINUTES'; MONTH: 'MONTH'; MONTHS: 'MONTHS'; MSCK: 'MSCK'; +NAMESPACE: 'NAMESPACE'; NAMESPACES: 'NAMESPACES'; NATURAL: 'NATURAL'; NO: 'NO'; @@ -1549,6 +1570,7 @@ POSITION: 'POSITION'; PRECEDING: 'PRECEDING'; PRIMARY: 'PRIMARY'; PRINCIPALS: 'PRINCIPALS'; +PROPERTIES: 'PROPERTIES'; PURGE: 'PURGE'; QUERY: 'QUERY'; RANGE: 'RANGE'; @@ -1622,6 +1644,7 @@ UNIQUE: 'UNIQUE'; UNKNOWN: 'UNKNOWN'; UNLOCK: 'UNLOCK'; UNSET: 'UNSET'; +UPDATE: 'UPDATE'; USE: 'USE'; USER: 'USER'; USING: 'USING'; 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 769cf36c3df3f..8ee90ed6f4c3b 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions; +import com.google.common.annotations.VisibleForTesting; + /** * Expression information, will be used to describe a expression. */ @@ -56,6 +58,11 @@ public String getArguments() { return arguments; } + @VisibleForTesting + public String getOriginalExamples() { + return examples; + } + public String getExamples() { return replaceFunctionName(examples); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java index 32ba24ff0256f..65e0b6be00ef3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java @@ -22,17 +22,17 @@ /** * An API to extend the Spark built-in session catalog. Implementation can get the built-in session - * catalog from {@link #setDelegateCatalog(TableCatalog)}, implement catalog functions with + * catalog from {@link #setDelegateCatalog(CatalogPlugin)}, implement catalog functions with * some custom logic and call the built-in session catalog at the end. For example, they can * implement {@code createTable}, do something else before calling {@code createTable} of the * built-in session catalog. */ @Experimental -public interface CatalogExtension extends TableCatalog { +public interface CatalogExtension extends TableCatalog, SupportsNamespaces { /** * This will be called only once by Spark to pass in the Spark built-in session catalog, after * {@link #initialize(String, CaseInsensitiveStringMap)} is called. */ - void setDelegateCatalog(TableCatalog delegate); + void setDelegateCatalog(CatalogPlugin delegate); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java index d7f0ee15f840e..b93acdc777e9c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; @@ -36,9 +37,9 @@ @Experimental public abstract class DelegatingCatalogExtension implements CatalogExtension { - private TableCatalog delegate; + private CatalogPlugin delegate; - public final void setDelegateCatalog(TableCatalog delegate) { + public final void setDelegateCatalog(CatalogPlugin delegate) { this.delegate = delegate; } @@ -52,22 +53,22 @@ public final void initialize(String name, CaseInsensitiveStringMap options) {} @Override public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { - return delegate.listTables(namespace); + return asTableCatalog().listTables(namespace); } @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - return delegate.loadTable(ident); + return asTableCatalog().loadTable(ident); } @Override public void invalidateTable(Identifier ident) { - delegate.invalidateTable(ident); + asTableCatalog().invalidateTable(ident); } @Override public boolean tableExists(Identifier ident) { - return delegate.tableExists(ident); + return asTableCatalog().tableExists(ident); } @Override @@ -76,25 +77,78 @@ public Table createTable( StructType schema, Transform[] partitions, Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { - return delegate.createTable(ident, schema, partitions, properties); + return asTableCatalog().createTable(ident, schema, partitions, properties); } @Override public Table alterTable( Identifier ident, TableChange... changes) throws NoSuchTableException { - return delegate.alterTable(ident, changes); + return asTableCatalog().alterTable(ident, changes); } @Override public boolean dropTable(Identifier ident) { - return delegate.dropTable(ident); + return asTableCatalog().dropTable(ident); } @Override public void renameTable( Identifier oldIdent, Identifier newIdent) throws NoSuchTableException, TableAlreadyExistsException { - delegate.renameTable(oldIdent, newIdent); + asTableCatalog().renameTable(oldIdent, newIdent); + } + + @Override + public String[] defaultNamespace() { + return asNamespaceCatalog().defaultNamespace(); + } + + @Override + public String[][] listNamespaces() throws NoSuchNamespaceException { + return asNamespaceCatalog().listNamespaces(); + } + + @Override + public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceException { + return asNamespaceCatalog().listNamespaces(namespace); + } + + @Override + public boolean namespaceExists(String[] namespace) { + return asNamespaceCatalog().namespaceExists(namespace); + } + + @Override + public Map loadNamespaceMetadata( + String[] namespace) throws NoSuchNamespaceException { + return asNamespaceCatalog().loadNamespaceMetadata(namespace); + } + + @Override + public void createNamespace( + String[] namespace, + Map metadata) throws NamespaceAlreadyExistsException { + asNamespaceCatalog().createNamespace(namespace, metadata); + } + + @Override + public void alterNamespace( + String[] namespace, + NamespaceChange... changes) throws NoSuchNamespaceException { + asNamespaceCatalog().alterNamespace(namespace, changes); + } + + @Override + public boolean dropNamespace(String[] namespace) throws NoSuchNamespaceException { + return asNamespaceCatalog().dropNamespace(namespace); + } + + private TableCatalog asTableCatalog() { + return (TableCatalog)delegate; + } + + private SupportsNamespaces asNamespaceCatalog() { + return (SupportsNamespaces)delegate; } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index 88ab51c1d70b3..20c22388b0ef9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -20,6 +20,9 @@ import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; +import java.util.Arrays; +import java.util.Objects; + /** * TableChange subclasses represent requested changes to a table. These are passed to * {@link TableCatalog#alterTable}. For example, @@ -210,6 +213,20 @@ public String property() { public String value() { return value; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SetProperty that = (SetProperty) o; + return property.equals(that.property) && + value.equals(that.value); + } + + @Override + public int hashCode() { + return Objects.hash(property, value); + } } /** @@ -227,6 +244,19 @@ private RemoveProperty(String property) { public String property() { return property; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RemoveProperty that = (RemoveProperty) o; + return property.equals(that.property); + } + + @Override + public int hashCode() { + return Objects.hash(property); + } } interface ColumnChange extends TableChange { @@ -269,6 +299,24 @@ public boolean isNullable() { public String comment() { return comment; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AddColumn addColumn = (AddColumn) o; + return isNullable == addColumn.isNullable && + Arrays.equals(fieldNames, addColumn.fieldNames) && + dataType.equals(addColumn.dataType) && + comment.equals(addColumn.comment); + } + + @Override + public int hashCode() { + int result = Objects.hash(dataType, isNullable, comment); + result = 31 * result + Arrays.hashCode(fieldNames); + return result; + } } /** @@ -296,6 +344,22 @@ public String[] fieldNames() { public String newName() { return newName; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RenameColumn that = (RenameColumn) o; + return Arrays.equals(fieldNames, that.fieldNames) && + newName.equals(that.newName); + } + + @Override + public int hashCode() { + int result = Objects.hash(newName); + result = 31 * result + Arrays.hashCode(fieldNames); + return result; + } } /** @@ -328,6 +392,23 @@ public DataType newDataType() { public boolean isNullable() { return isNullable; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + UpdateColumnType that = (UpdateColumnType) o; + return isNullable == that.isNullable && + Arrays.equals(fieldNames, that.fieldNames) && + newDataType.equals(that.newDataType); + } + + @Override + public int hashCode() { + int result = Objects.hash(newDataType, isNullable); + result = 31 * result + Arrays.hashCode(fieldNames); + return result; + } } /** @@ -354,6 +435,22 @@ public String[] fieldNames() { public String newComment() { return newComment; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + UpdateColumnComment that = (UpdateColumnComment) o; + return Arrays.equals(fieldNames, that.fieldNames) && + newComment.equals(that.newComment); + } + + @Override + public int hashCode() { + int result = Objects.hash(newComment); + result = 31 * result + Arrays.hashCode(fieldNames); + return result; + } } /** @@ -372,6 +469,19 @@ private DeleteColumn(String[] fieldNames) { public String[] fieldNames() { return fieldNames; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DeleteColumn that = (DeleteColumn) o; + return Arrays.equals(fieldNames, that.fieldNames); + } + + @Override + public int hashCode() { + return Arrays.hashCode(fieldNames); + } } } 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 494387ecfe0c1..3f9d07520e05f 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 @@ -17,12 +17,24 @@ package org.apache.spark.sql +import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} +import java.util.{Base64, TimeZone} + import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.util.hashing.MurmurHash3 -import org.apache.spark.annotation.Stable +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.{Private, Stable, Unstable} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, Decimal, MapType, StringType, StructType, UserDefinedType} /** * @since 1.3.0 @@ -501,4 +513,96 @@ trait Row extends Serializable { private def getAnyValAs[T <: AnyVal](i: Int): T = if (isNullAt(i)) throw new NullPointerException(s"Value at index $i is null") else getAs[T](i) + + /** + * The compact JSON representation of this row. + * @since 3.0 + */ + @Unstable + def json: String = compact(jsonValue) + + /** + * The pretty (i.e. indented) JSON representation of this row. + * @since 3.0 + */ + @Unstable + def prettyJson: String = pretty(render(jsonValue)) + + /** + * JSON representation of the row. + * + * Note that this only supports the data types that are also supported by + * [[org.apache.spark.sql.catalyst.encoders.RowEncoder]]. + * + * @return the JSON representation of the row. + */ + private[sql] def jsonValue: JValue = { + require(schema != null, "JSON serialization requires a non-null schema.") + + lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + lazy val dateFormatter = DateFormatter.apply(zoneId) + lazy val timestampFormatter = TimestampFormatter(zoneId) + + // Convert an iterator of values to a json array + def iteratorToJsonArray(iterator: Iterator[_], elementType: DataType): JArray = { + JArray(iterator.map(toJson(_, elementType)).toList) + } + + // Convert a value to json. + def toJson(value: Any, dataType: DataType): JValue = (value, dataType) match { + case (null, _) => JNull + case (b: Boolean, _) => JBool(b) + case (b: Byte, _) => JLong(b) + case (s: Short, _) => JLong(s) + case (i: Int, _) => JLong(i) + case (l: Long, _) => JLong(l) + case (f: Float, _) => JDouble(f) + case (d: Double, _) => JDouble(d) + case (d: BigDecimal, _) => JDecimal(d) + case (d: java.math.BigDecimal, _) => JDecimal(d) + case (d: Decimal, _) => JDecimal(d.toBigDecimal) + case (s: String, _) => JString(s) + case (b: Array[Byte], BinaryType) => + JString(Base64.getEncoder.encodeToString(b)) + case (d: LocalDate, _) => + JString(dateFormatter.format(DateTimeUtils.localDateToDays(d))) + case (d: Date, _) => + JString(dateFormatter.format(DateTimeUtils.fromJavaDate(d))) + case (i: Instant, _) => + JString(timestampFormatter.format(DateTimeUtils.instantToMicros(i))) + case (t: Timestamp, _) => + JString(timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t))) + case (a: Array[_], ArrayType(elementType, _)) => + iteratorToJsonArray(a.iterator, elementType) + case (s: Seq[_], ArrayType(elementType, _)) => + iteratorToJsonArray(s.iterator, elementType) + case (m: Map[String @unchecked, _], MapType(StringType, valueType, _)) => + new JObject(m.toList.sortBy(_._1).map { + case (k, v) => k -> toJson(v, valueType) + }) + case (m: Map[_, _], MapType(keyType, valueType, _)) => + new JArray(m.iterator.map { + case (k, v) => + new JObject("key" -> toJson(k, keyType) :: "value" -> toJson(v, valueType) :: Nil) + }.toList) + case (r: Row, _) => r.jsonValue + case (v: Any, udt: UserDefinedType[Any @unchecked]) => + val dataType = udt.sqlType + toJson(CatalystTypeConverters.convertToScala(udt.serialize(v), dataType), dataType) + case _ => + throw new IllegalArgumentException(s"Failed to convert value $value " + + s"(class of ${value.getClass}}) with the type of $dataType to JSON.") + } + + // Convert the row fields to json + var n = 0 + var elements = new mutable.ListBuffer[JField] + val len = length + while (n < len) { + val field = schema(n) + elements += (field.name -> toJson(apply(n), field.dataType)) + n += 1 + } + new JObject(elements.toList) + } } 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 bdab407688a65..1bb13e3368c02 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 @@ -56,7 +56,7 @@ abstract class InternalRow extends SpecializedGetters with Serializable { * Note: In order to support update decimal with precision > 18 in UnsafeRow, * CAN NOT call setNullAt() for decimal column on UnsafeRow, call setDecimal(i, null, precision). */ - def setDecimal(i: Int, value: Decimal, precision: Int) { update(i, value) } + def setDecimal(i: Int, value: Decimal, precision: Int): Unit = update(i, value) /** * Make a copy of the current [[InternalRow]] object. 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 7c81185388d02..21bf926af50d7 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 @@ -34,7 +34,6 @@ 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._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL @@ -52,12 +51,15 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * to resolve attribute references. */ object SimpleAnalyzer extends Analyzer( - new SessionCatalog( - new InMemoryCatalog, - EmptyFunctionRegistry, - new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) { - override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean) {} - }, + new CatalogManager( + new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true), + FakeV2SessionCatalog, + new SessionCatalog( + new InMemoryCatalog, + EmptyFunctionRegistry, + new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) { + override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = {} + }), new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) object FakeV2SessionCatalog extends TableCatalog { @@ -75,7 +77,7 @@ object FakeV2SessionCatalog extends TableCatalog { override def dropTable(ident: Identifier): Boolean = fail() override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = fail() override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = fail() - override def name(): String = fail() + override def name(): String = CatalogManager.SESSION_CATALOG_NAME } /** @@ -118,23 +120,27 @@ object AnalysisContext { * [[UnresolvedRelation]]s into fully typed objects using information in a [[SessionCatalog]]. */ class Analyzer( - catalog: SessionCatalog, - v2SessionCatalog: TableCatalog, + override val catalogManager: CatalogManager, conf: SQLConf, maxIterations: Int) extends RuleExecutor[LogicalPlan] with CheckAnalysis with LookupCatalog { + private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog + + override def isView(nameParts: Seq[String]): Boolean = v1SessionCatalog.isView(nameParts) + // Only for tests. def this(catalog: SessionCatalog, conf: SQLConf) = { - this(catalog, FakeV2SessionCatalog, conf, conf.optimizerMaxIterations) + this( + new CatalogManager(conf, FakeV2SessionCatalog, catalog), + conf, + conf.optimizerMaxIterations) } - def this(catalog: SessionCatalog, v2SessionCatalog: TableCatalog, conf: SQLConf) = { - this(catalog, v2SessionCatalog, conf, conf.optimizerMaxIterations) + def this(catalogManager: CatalogManager, conf: SQLConf) = { + this(catalogManager, conf, conf.optimizerMaxIterations) } - override val catalogManager: CatalogManager = new CatalogManager(conf, v2SessionCatalog) - def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { AnalysisHelper.markInAnalyzer { val analyzed = executeAndTrack(plan, tracker) @@ -180,8 +186,7 @@ class Analyzer( lazy val batches: Seq[Batch] = Seq( Batch("Hints", fixedPoint, new ResolveHints.ResolveJoinStrategyHints(conf), - ResolveHints.ResolveCoalesceHints, - new ResolveHints.RemoveAllHints(conf)), + ResolveHints.ResolveCoalesceHints), Batch("Simple Sanity Check", Once, LookupFunctions), Batch("Substitution", fixedPoint, @@ -191,8 +196,7 @@ class Analyzer( new SubstituteUnresolvedOrdinals(conf)), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: - ResolveAlterTable :: - ResolveDescribeTable :: + new ResolveCatalogs(catalogManager) :: ResolveInsertInto :: ResolveTables :: ResolveRelations :: @@ -221,13 +225,16 @@ class Analyzer( ResolveAggregateFunctions :: TimeWindowing :: ResolveInlineTables(conf) :: - ResolveHigherOrderFunctions(catalog) :: + ResolveHigherOrderFunctions(v1SessionCatalog) :: ResolveLambdaVariables(conf) :: ResolveTimeZone(conf) :: ResolveRandomSeed :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), + Batch("PostgreSQL Dialect", Once, PostgreSQLDialect.postgreSQLDialectRules: _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), + Batch("Remove Unresolved Hints", Once, + new ResolveHints.RemoveAllHints(conf)), Batch("Nondeterministic", Once, PullOutNondeterministic), Batch("UDF", Once, @@ -673,6 +680,11 @@ class Analyzer( lookupV2Relation(u.multipartIdentifier) .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) + + case u: UnresolvedV2Relation => + CatalogV2Util.loadTable(u.catalog, u.tableName).map { table => + DataSourceV2Relation.create(table) + }.getOrElse(u) } } @@ -709,7 +721,7 @@ class Analyzer( // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) - if catalog.isTemporaryTable(ident) => + if v1SessionCatalog.isTemporaryTable(ident) => resolveRelation(lookupTableFromCatalog(ident, u, AnalysisContext.get.defaultDatabase)) case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => @@ -766,7 +778,7 @@ class Analyzer( val tableIdentWithDb = tableIdentifier.copy( database = tableIdentifier.database.orElse(defaultDatabase)) try { - catalog.lookupRelation(tableIdentWithDb) + v1SessionCatalog.lookupRelation(tableIdentWithDb) } catch { case _: NoSuchTableException | _: NoSuchDatabaseException => u @@ -780,8 +792,9 @@ class Analyzer( // Note that we are testing (!db_exists || !table_exists) because the catalog throws // an exception from tableExists if the database does not exist. private def isRunningDirectlyOnFiles(table: TableIdentifier): Boolean = { - table.database.isDefined && conf.runSQLonFile && !catalog.isTemporaryTable(table) && - (!catalog.databaseExists(table.database.get) || !catalog.tableExists(table)) + table.database.isDefined && conf.runSQLonFile && !v1SessionCatalog.isTemporaryTable(table) && + (!v1SessionCatalog.databaseExists(table.database.get) + || !v1SessionCatalog.tableExists(table)) } } @@ -903,82 +916,6 @@ 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.connector.catalog.CatalogV2Implicits._ - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case alter @ AlterTableAddColumnsStatement(tableName, cols) => - val changes = cols.map { col => - TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) - } - resolveV2Alter(tableName, changes).getOrElse(alter) - - case alter @ AlterTableAlterColumnStatement(tableName, colName, dataType, comment) => - val typeChange = dataType.map { newDataType => - TableChange.updateColumnType(colName.toArray, newDataType, true) - } - - val commentChange = comment.map { newComment => - TableChange.updateColumnComment(colName.toArray, newComment) - } - - resolveV2Alter(tableName, typeChange.toSeq ++ commentChange.toSeq).getOrElse(alter) - - case alter @ AlterTableRenameColumnStatement(tableName, col, newName) => - val changes = Seq(TableChange.renameColumn(col.toArray, newName)) - resolveV2Alter(tableName, changes).getOrElse(alter) - - case alter @ AlterTableDropColumnsStatement(tableName, cols) => - val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) - resolveV2Alter(tableName, changes).getOrElse(alter) - - case alter @ AlterTableSetPropertiesStatement(tableName, props) => - val changes = props.map { case (key, value) => - TableChange.setProperty(key, value) - } - - resolveV2Alter(tableName, changes.toSeq).getOrElse(alter) - - case alter @ AlterTableUnsetPropertiesStatement(tableName, keys, _) => - resolveV2Alter(tableName, keys.map(key => TableChange.removeProperty(key))).getOrElse(alter) - - case alter @ AlterTableSetLocationStatement(tableName, newLoc) => - resolveV2Alter(tableName, Seq(TableChange.setProperty("location", newLoc))).getOrElse(alter) - } - - private def resolveV2Alter( - tableName: Seq[String], - changes: Seq[TableChange]): Option[AlterTable] = { - lookupV2RelationAndCatalog(tableName).map { - case (relation, catalog, ident) => - AlterTable(catalog.asTableCatalog, ident, relation, changes) - } - } - } - - /** - * Resolve DESCRIBE TABLE statements that use a DSv2 catalog. - * - * This rule converts unresolved DESCRIBE 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 ResolveDescribeTable extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case describe @ DescribeTableStatement( - CatalogObjectIdentifier(Some(v2Catalog), ident), _, isExtended) => - DescribeTable(UnresolvedRelation(describe.tableName), isExtended) - } - } - /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from * a logical plan node's children. @@ -1575,13 +1512,14 @@ class Analyzer( plan.resolveExpressions { case f: UnresolvedFunction if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f - case f: UnresolvedFunction if catalog.isRegisteredFunction(f.name) => f - case f: UnresolvedFunction if catalog.isPersistentFunction(f.name) => + case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f + case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) => externalFunctionNameSet.add(normalizeFuncName(f.name)) f case f: UnresolvedFunction => withPosition(f) { - throw new NoSuchFunctionException(f.name.database.getOrElse(catalog.getCurrentDatabase), + throw new NoSuchFunctionException( + f.name.database.getOrElse(v1SessionCatalog.getCurrentDatabase), f.name.funcName) } } @@ -1596,7 +1534,7 @@ class Analyzer( val databaseName = name.database match { case Some(a) => formatDatabaseName(a) - case None => catalog.getCurrentDatabase + case None => v1SessionCatalog.getCurrentDatabase } FunctionIdentifier(funcName, Some(databaseName)) @@ -1621,7 +1559,7 @@ class Analyzer( } case u @ UnresolvedGenerator(name, children) => withPosition(u) { - catalog.lookupFunction(name, children) match { + v1SessionCatalog.lookupFunction(name, children) match { case generator: Generator => generator case other => failAnalysis(s"$name is expected to be a generator. However, " + @@ -1630,7 +1568,7 @@ class Analyzer( } case u @ UnresolvedFunction(funcId, children, isDistinct) => withPosition(u) { - catalog.lookupFunction(funcId, children) match { + v1SessionCatalog.lookupFunction(funcId, children) match { // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within // the context of a Window clause. They do not need to be wrapped in an // AggregateExpression. @@ -1759,8 +1697,10 @@ class Analyzer( // Only a few unary nodes (Project/Filter/Aggregate) can contain subqueries. case q: UnaryNode if q.childrenResolved => resolveSubQueries(q, q.children) - case d: DeleteFromTable if d.childrenResolved => - resolveSubQueries(d, d.children) + case j: Join if j.childrenResolved => + resolveSubQueries(j, Seq(j, j.left, j.right)) + case s: SupportsSubquery if s.childrenResolved => + resolveSubQueries(s, s.children) } } @@ -2566,9 +2506,9 @@ class Analyzer( override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { case append @ AppendData(table, query, _, isByName) if table.resolved && query.resolved && !append.outputResolved => + validateStoreAssignmentPolicy() val projection = - TableOutputResolver.resolveOutputColumns( - table.name, table.output, query, isByName, conf, storeAssignmentPolicy) + TableOutputResolver.resolveOutputColumns(table.name, table.output, query, isByName, conf) if (projection != query) { append.copy(query = projection) @@ -2578,9 +2518,9 @@ class Analyzer( case overwrite @ OverwriteByExpression(table, _, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => + validateStoreAssignmentPolicy() val projection = - TableOutputResolver.resolveOutputColumns( - table.name, table.output, query, isByName, conf, storeAssignmentPolicy) + TableOutputResolver.resolveOutputColumns(table.name, table.output, query, isByName, conf) if (projection != query) { overwrite.copy(query = projection) @@ -2590,9 +2530,9 @@ class Analyzer( case overwrite @ OverwritePartitionsDynamic(table, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => + validateStoreAssignmentPolicy() val projection = - TableOutputResolver.resolveOutputColumns( - table.name, table.output, query, isByName, conf, storeAssignmentPolicy) + TableOutputResolver.resolveOutputColumns(table.name, table.output, query, isByName, conf) if (projection != query) { overwrite.copy(query = projection) @@ -2602,16 +2542,14 @@ class Analyzer( } } - private def storeAssignmentPolicy: StoreAssignmentPolicy.Value = { - val policy = conf.storeAssignmentPolicy.getOrElse(StoreAssignmentPolicy.STRICT) + private def validateStoreAssignmentPolicy(): Unit = { // SPARK-28730: LEGACY store assignment policy is disallowed in data source v2. - if (policy == StoreAssignmentPolicy.LEGACY) { + if (conf.storeAssignmentPolicy == StoreAssignmentPolicy.LEGACY) { val configKey = SQLConf.STORE_ASSIGNMENT_POLICY.key throw new AnalysisException(s""" |"LEGACY" store assignment policy is disallowed in Spark data source V2. |Please set the configuration $configKey to other values.""".stripMargin) } - policy } private def commonNaturalJoinProcessing( @@ -2831,17 +2769,17 @@ class Analyzer( private def lookupV2RelationAndCatalog( identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = identifier match { - case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => None - case CatalogObjectIdentifier(Some(v2Catalog), ident) => - CatalogV2Util.loadTable(v2Catalog, ident) match { - case Some(table) => Some((DataSourceV2Relation.create(table), v2Catalog, ident)) + case AsTemporaryViewIdentifier(ti) if v1SessionCatalog.isTemporaryTable(ti) => None + case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => + CatalogV2Util.loadTable(catalog, ident) match { + case Some(table) => Some((DataSourceV2Relation.create(table), catalog, ident)) case None => None } - case CatalogObjectIdentifier(None, ident) => - CatalogV2Util.loadTable(catalogManager.v2SessionCatalog, ident) match { + case CatalogObjectIdentifier(catalog, ident) if CatalogV2Util.isSessionCatalog(catalog) => + CatalogV2Util.loadTable(catalog, ident) match { case Some(_: V1Table) => None case Some(table) => - Some((DataSourceV2Relation.create(table), catalogManager.v2SessionCatalog, ident)) + Some((DataSourceV2Relation.create(table), catalog, ident)) case None => None } case _ => None 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 4a19806bd80ff..d9dc9ebbcaf3c 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 @@ -24,7 +24,6 @@ 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.catalyst.plans.logical.sql.{AlterTableStatement, InsertIntoStatement} import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -34,6 +33,8 @@ import org.apache.spark.sql.types._ */ trait CheckAnalysis extends PredicateHelper { + protected def isView(nameParts: Seq[String]): Boolean + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** @@ -96,6 +97,13 @@ trait CheckAnalysis extends PredicateHelper { case InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) => failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + case u: UnresolvedV2Relation if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case u: UnresolvedV2Relation => + u.failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, @@ -357,9 +365,6 @@ trait CheckAnalysis extends PredicateHelper { case _ => } - case alter: AlterTableStatement => - alter.failAnalysis(s"Table or view not found: ${alter.tableName.quoted}") - case alter: AlterTable if alter.childrenResolved => val table = alter.table def findField(operation: String, fieldName: Array[String]): StructField = { @@ -587,19 +592,19 @@ trait CheckAnalysis extends PredicateHelper { // Only certain operators are allowed to host subquery expression containing // outer references. plan match { - case _: Filter | _: Aggregate | _: Project | _: DeleteFromTable => // Ok + case _: Filter | _: Aggregate | _: Project | _: SupportsSubquery => // Ok case other => failAnalysis( "Correlated scalar sub-queries can only be used in a " + - s"Filter/Aggregate/Project: $plan") + s"Filter/Aggregate/Project and a few commands: $plan") } } case inSubqueryOrExistsSubquery => plan match { - case _: Filter | _: DeleteFromTable => // Ok + case _: Filter | _: SupportsSubquery | _: Join => // Ok case _ => failAnalysis(s"IN/EXISTS predicate sub-queries can only be used in" + - s" Filter/DeleteFromTable: $plan") + s" Filter/Join and a few commands: $plan") } } 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 d5728b9027579..04e8963944fda 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 @@ -314,8 +314,10 @@ object FunctionRegistry { expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), expression[EveryAgg]("every"), + expression[EveryAgg]("bool_and"), expression[AnyAgg]("any"), - expression[SomeAgg]("some"), + expression[AnyAgg]("some"), + expression[AnyAgg]("bool_or"), // string functions expression[Ascii]("ascii"), @@ -522,6 +524,9 @@ object FunctionRegistry { expression[BitwiseNot]("~"), expression[BitwiseOr]("|"), expression[BitwiseXor]("^"), + expression[BitwiseCount]("bit_count"), + expression[BitAndAgg]("bit_and"), + expression[BitOrAgg]("bit_or"), // json expression[StructsToJson]("to_json"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala new file mode 100644 index 0000000000000..934e53703e241 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.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.analysis + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastStringToBoolean +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BooleanType, StringType} + +object PostgreSQLDialect { + val postgreSQLDialectRules: List[Rule[LogicalPlan]] = + CastStringToBoolean :: + Nil + + object CastStringToBoolean extends Rule[LogicalPlan] with Logging { + override def apply(plan: LogicalPlan): LogicalPlan = { + // The SQL configuration `spark.sql.dialect` can be changed in runtime. + // To make sure the configuration is effective, we have to check it during rule execution. + val conf = SQLConf.get + if (conf.usePostgreSQLDialect) { + plan.transformExpressions { + case Cast(child, dataType, _) + if dataType == BooleanType && child.dataType == StringType => + PostgreCastStringToBoolean(child) + } + } else { + plan + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala new file mode 100644 index 0000000000000..9803fda0678ff --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.AnalysisException +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange} + +/** + * Resolves catalogs from the multi-part identifiers in SQL statements, and convert the statements + * to the corresponding v2 commands if the resolved catalog is not the session catalog. + */ +class ResolveCatalogs(val catalogManager: CatalogManager) + extends Rule[LogicalPlan] with LookupCatalog { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case AlterTableAddColumnsStatement( + nameParts @ NonSessionCatalog(catalog, tableName), cols) => + val changes = cols.map { col => + TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + } + createAlterTable(nameParts, catalog, tableName, changes) + + case AlterTableAlterColumnStatement( + nameParts @ NonSessionCatalog(catalog, tableName), colName, dataType, comment) => + val typeChange = dataType.map { newDataType => + TableChange.updateColumnType(colName.toArray, newDataType, true) + } + val commentChange = comment.map { newComment => + TableChange.updateColumnComment(colName.toArray, newComment) + } + createAlterTable(nameParts, catalog, tableName, typeChange.toSeq ++ commentChange) + + case AlterTableRenameColumnStatement( + nameParts @ NonSessionCatalog(catalog, tableName), col, newName) => + val changes = Seq(TableChange.renameColumn(col.toArray, newName)) + createAlterTable(nameParts, catalog, tableName, changes) + + case AlterTableDropColumnsStatement( + nameParts @ NonSessionCatalog(catalog, tableName), cols) => + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + createAlterTable(nameParts, catalog, tableName, changes) + + case AlterTableSetPropertiesStatement( + nameParts @ NonSessionCatalog(catalog, tableName), props) => + val changes = props.map { case (key, value) => + TableChange.setProperty(key, value) + }.toSeq + createAlterTable(nameParts, catalog, tableName, changes) + + // TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag. + case AlterTableUnsetPropertiesStatement( + nameParts @ NonSessionCatalog(catalog, tableName), keys, _) => + val changes = keys.map(key => TableChange.removeProperty(key)) + createAlterTable(nameParts, catalog, tableName, changes) + + case AlterTableSetLocationStatement( + nameParts @ NonSessionCatalog(catalog, tableName), newLoc) => + val changes = Seq(TableChange.setProperty("location", newLoc)) + createAlterTable(nameParts, catalog, tableName, changes) + + case AlterViewSetPropertiesStatement( + NonSessionCatalog(catalog, tableName), props) => + throw new AnalysisException( + s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + + s"because view support in catalog has not been implemented yet") + + case AlterViewUnsetPropertiesStatement( + NonSessionCatalog(catalog, tableName), keys, ifExists) => + throw new AnalysisException( + s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + + s"because view support in catalog has not been implemented yet") + + case DeleteFromStatement( + nameParts @ NonSessionCatalog(catalog, tableName), tableAlias, condition) => + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + val aliased = tableAlias.map(SubqueryAlias(_, r)).getOrElse(r) + DeleteFromTable(aliased, condition) + + case u @ UpdateTableStatement( + nameParts @ CatalogAndIdentifierParts(catalog, tableName), _, _, _, _) => + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + val aliased = u.tableAlias.map(SubqueryAlias(_, r)).getOrElse(r) + val columns = u.columns.map(UnresolvedAttribute(_)) + UpdateTable(aliased, columns, u.values, u.condition) + + case DescribeTableStatement( + nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => + if (partitionSpec.nonEmpty) { + throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.") + } + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + DescribeTable(r, isExtended) + + case DescribeColumnStatement( + NonSessionCatalog(catalog, tableName), colNameParts, isExtended) => + throw new AnalysisException("Describing columns is not supported for v2 tables.") + + case c @ CreateTableStatement( + NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + CreateV2Table( + catalog.asTableCatalog, + tableName.asIdentifier, + c.tableSchema, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + ignoreIfExists = c.ifNotExists) + + case c @ CreateTableAsSelectStatement( + NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + CreateTableAsSelect( + catalog.asTableCatalog, + tableName.asIdentifier, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + c.asSelect, + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + writeOptions = c.options.filterKeys(_ != "path"), + ignoreIfExists = c.ifNotExists) + + case RefreshTableStatement(NonSessionCatalog(catalog, tableName)) => + RefreshTable(catalog.asTableCatalog, tableName.asIdentifier) + + case c @ ReplaceTableStatement( + NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + ReplaceTable( + catalog.asTableCatalog, + tableName.asIdentifier, + c.tableSchema, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + orCreate = c.orCreate) + + case c @ ReplaceTableAsSelectStatement( + NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + ReplaceTableAsSelect( + catalog.asTableCatalog, + tableName.asIdentifier, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + c.asSelect, + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + writeOptions = c.options.filterKeys(_ != "path"), + orCreate = c.orCreate) + + case DropTableStatement(NonSessionCatalog(catalog, tableName), ifExists, _) => + DropTable(catalog.asTableCatalog, tableName.asIdentifier, ifExists) + + case DropViewStatement(NonSessionCatalog(catalog, viewName), _) => + throw new AnalysisException( + s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " + + s"because view support in catalog has not been implemented yet") + + case c @ CreateNamespaceStatement(NonSessionCatalog(catalog, nameParts), _, _) => + CreateNamespace( + catalog.asNamespaceCatalog, + nameParts, + c.ifNotExists, + c.properties) + + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => + ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) + + case ShowNamespacesStatement(None, pattern) => + ShowNamespaces(currentCatalog.asNamespaceCatalog, None, pattern) + + case ShowTablesStatement(Some(NonSessionCatalog(catalog, nameParts)), pattern) => + ShowTables(catalog.asTableCatalog, nameParts, pattern) + + case ShowTablesStatement(None, pattern) if !isSessionCatalog(currentCatalog) => + ShowTables(currentCatalog.asTableCatalog, catalogManager.currentNamespace, pattern) + + case UseStatement(isNamespaceSet, nameParts) => + if (isNamespaceSet) { + SetCatalogAndNamespace(catalogManager, None, Some(nameParts)) + } else { + val CatalogAndNamespace(catalog, namespace) = nameParts + SetCatalogAndNamespace(catalogManager, Some(catalog.name()), namespace) + } + } + + object NonSessionCatalog { + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { + case CatalogAndIdentifierParts(catalog, parts) if !isSessionCatalog(catalog) => + Some(catalog -> parts) + case _ => None + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala index 6769773cfec45..4f33ca99c02db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, AnsiCast, Attribute, Cast, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy @@ -32,8 +32,7 @@ object TableOutputResolver { expected: Seq[Attribute], query: LogicalPlan, byName: Boolean, - conf: SQLConf, - storeAssignmentPolicy: StoreAssignmentPolicy.Value): LogicalPlan = { + conf: SQLConf): LogicalPlan = { if (expected.size < query.output.size) { throw new AnalysisException( @@ -47,8 +46,7 @@ object TableOutputResolver { expected.flatMap { tableAttr => query.resolve(Seq(tableAttr.name), conf.resolver) match { case Some(queryExpr) => - checkField( - tableAttr, queryExpr, byName, conf, storeAssignmentPolicy, err => errors += err) + checkField(tableAttr, queryExpr, byName, conf, err => errors += err) case None => errors += s"Cannot find data for output column '${tableAttr.name}'" None @@ -66,8 +64,7 @@ object TableOutputResolver { query.output.zip(expected).flatMap { case (queryExpr, tableAttr) => - checkField( - tableAttr, queryExpr, byName, conf, storeAssignmentPolicy, err => errors += err) + checkField(tableAttr, queryExpr, byName, conf, err => errors += err) } } @@ -88,9 +85,9 @@ object TableOutputResolver { queryExpr: NamedExpression, byName: Boolean, conf: SQLConf, - storeAssignmentPolicy: StoreAssignmentPolicy.Value, addError: String => Unit): Option[NamedExpression] = { + val storeAssignmentPolicy = conf.storeAssignmentPolicy lazy val outputField = if (tableAttr.dataType.sameType(queryExpr.dataType) && tableAttr.name == queryExpr.name && tableAttr.metadata == queryExpr.metadata) { @@ -99,9 +96,16 @@ object TableOutputResolver { // Renaming is needed for handling the following cases like // 1) Column names/types do not match, e.g., INSERT INTO TABLE tab1 SELECT 1, 2 // 2) Target tables have column metadata - Some(Alias( - Cast(queryExpr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)), - tableAttr.name)(explicitMetadata = Option(tableAttr.metadata))) + storeAssignmentPolicy match { + case StoreAssignmentPolicy.ANSI => + Some(Alias( + AnsiCast(queryExpr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)), + tableAttr.name)(explicitMetadata = Option(tableAttr.metadata))) + case _ => + Some(Alias( + Cast(queryExpr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)), + tableAttr.name)(explicitMetadata = Option(tableAttr.metadata))) + } } storeAssignmentPolicy match { 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 8ea6e1b0f1808..c451eb2b877da 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 @@ -677,8 +677,9 @@ 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) => + val preferIntegralDivision = conf.usePostgreSQLDialect (left.dataType, right.dataType) match { - case (_: IntegralType, _: IntegralType) if conf.preferIntegralDivision => + case (_: IntegralType, _: IntegralType) if preferIntegralDivision => IntegralDivide(left, right) case _ => Divide(Cast(left, DoubleType), Cast(right, DoubleType)) @@ -827,8 +828,9 @@ object TypeCoercion { /** * 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. + * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType + * and TimestampType/IntegerType/DateType to DateAdd/DateSub/SubtractDates and + * to SubtractTimestamps. */ object DateTimeOperations extends Rule[LogicalPlan] { @@ -848,7 +850,14 @@ object TypeCoercion { 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) + case Subtract(l @ DateType(), r @ DateType()) => + if (SQLConf.get.usePostgreSQLDialect) DateDiff(l, r) else SubtractDates(l, r) + case Subtract(l @ TimestampType(), r @ TimestampType()) => + SubtractTimestamps(l, r) + case Subtract(l @ TimestampType(), r @ DateType()) => + SubtractTimestamps(l, Cast(r, TimestampType)) + case Subtract(l @ DateType(), r @ TimestampType()) => + SubtractTimestamps(Cast(l, TimestampType), r) } } 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 288ff1a04737e..2f8cb26ffaa9c 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, CurrentDate, CurrentTimestamp, MonotonicallyIncreasingID} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -30,7 +31,7 @@ import org.apache.spark.sql.streaming.OutputMode /** * Analyzes the presence of unsupported operations in a logical plan. */ -object UnsupportedOperationChecker { +object UnsupportedOperationChecker extends Logging { def checkForBatch(plan: LogicalPlan): Unit = { plan.foreachUp { @@ -41,8 +42,50 @@ object UnsupportedOperationChecker { } } - def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = { + def checkStreamingQueryGlobalWatermarkLimit( + plan: LogicalPlan, + outputMode: OutputMode, + failWhenDetected: Boolean): Unit = { + def isStatefulOperationPossiblyEmitLateRows(p: LogicalPlan): Boolean = p match { + case s: Aggregate + if s.isStreaming && outputMode == InternalOutputModes.Append => true + case Join(left, right, joinType, _, _) + if left.isStreaming && right.isStreaming && joinType != Inner => true + case f: FlatMapGroupsWithState + if f.isStreaming && f.outputMode == OutputMode.Append() => true + case _ => false + } + + def isStatefulOperation(p: LogicalPlan): Boolean = p match { + case s: Aggregate if s.isStreaming => true + case _ @ Join(left, right, _, _, _) if left.isStreaming && right.isStreaming => true + case f: FlatMapGroupsWithState if f.isStreaming => true + case d: Deduplicate if d.isStreaming => true + case _ => false + } + + try { + plan.foreach { subPlan => + if (isStatefulOperation(subPlan)) { + subPlan.find { p => + (p ne subPlan) && isStatefulOperationPossiblyEmitLateRows(p) + }.foreach { _ => + val errorMsg = "Detected pattern of possible 'correctness' issue " + + "due to global watermark. " + + "The query contains stateful operation which can emit rows older than " + + "the current watermark plus allowed late record delay, which are \"late rows\"" + + " in downstream stateful operations and these rows can be discarded. " + + "Please refer the programming guide doc for more details." + throwError(errorMsg)(plan) + } + } + } + } catch { + case e: AnalysisException if !failWhenDetected => logWarning(s"${e.message};\n$plan") + } + } + def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = { if (!plan.isStreaming) { throwError( "Queries without streaming sources cannot be executed with writeStream.start()")(plan) @@ -339,6 +382,8 @@ object UnsupportedOperationChecker { // Check if there are unsupported expressions in streaming query plan. checkUnsupportedExpressions(subPlan) } + + checkStreamingQueryGlobalWatermarkLimit(plan, outputMode, failWhenDetected = false) } def checkForContinuous(plan: LogicalPlan, outputMode: OutputMode): Unit = { 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 40f25fb7f9728..e5a6f30c330e6 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 @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.parser.ParserUtils import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.types.{DataType, Metadata, StructType} /** @@ -59,6 +60,28 @@ object UnresolvedRelation { UnresolvedRelation(tableIdentifier.database.toSeq :+ tableIdentifier.table) } +/** + * A variant of [[UnresolvedRelation]] which can only be resolved to a v2 relation + * (`DataSourceV2Relation`), not v1 relation or temp view. + * + * @param originalNameParts the original table identifier name parts before catalog is resolved. + * @param catalog The catalog which the table should be looked up from. + * @param tableName The name of the table to look up. + */ +case class UnresolvedV2Relation( + originalNameParts: Seq[String], + catalog: TableCatalog, + tableName: Identifier) + extends LeafNode with NamedRelation { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override def name: String = originalNameParts.quoted + + override def output: Seq[Attribute] = Nil + + override lazy val resolved = false +} + /** * 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]]. 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 2a6124a4079a1..be8526454f9f1 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 @@ -353,9 +353,20 @@ class SessionCatalog( val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableDefinition.identifier.table) val tableIdentifier = TableIdentifier(table, Some(db)) - val newTableDefinition = tableDefinition.copy(identifier = tableIdentifier) requireDbExists(db) requireTableExists(tableIdentifier) + val newTableDefinition = if (tableDefinition.storage.locationUri.isDefined + && !tableDefinition.storage.locationUri.get.isAbsolute) { + // make the location of the table qualified. + val qualifiedTableLocation = + makeQualifiedPath(tableDefinition.storage.locationUri.get) + tableDefinition.copy( + storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), + identifier = tableIdentifier) + } else { + tableDefinition.copy(identifier = tableIdentifier) + } + externalCatalog.alterTable(newTableDefinition) } @@ -770,6 +781,18 @@ class SessionCatalog( } } + def isView(nameParts: Seq[String]): Boolean = { + nameParts.length <= 2 && { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + val ident = nameParts.asTableIdentifier + try { + getTempViewOrPermanentTableMetadata(ident).tableType == CatalogTableType.VIEW + } catch { + case _: NoSuchTableException => false + } + } + } + /** * List all tables in the specified database, including local temporary views. * @@ -882,7 +905,8 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) requireNonEmptyValueInPartitionSpec(parts.map(_.spec)) - externalCatalog.createPartitions(db, table, parts, ignoreIfExists) + externalCatalog.createPartitions( + db, table, partitionWithQualifiedPath(tableName, parts), ignoreIfExists) } /** @@ -942,7 +966,7 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) requireNonEmptyValueInPartitionSpec(parts.map(_.spec)) - externalCatalog.alterPartitions(db, table, parts) + externalCatalog.alterPartitions(db, table, partitionWithQualifiedPath(tableName, parts)) } /** @@ -1064,6 +1088,23 @@ class SessionCatalog( } } + /** + * Make the partition path qualified. + * If the partition path is relative, e.g. 'paris', it will be qualified with + * parent path using table location, e.g. 'file:/warehouse/table/paris' + */ + private def partitionWithQualifiedPath( + tableIdentifier: TableIdentifier, + parts: Seq[CatalogTablePartition]): Seq[CatalogTablePartition] = { + lazy val tbl = getTableMetadata(tableIdentifier) + parts.map { part => + if (part.storage.locationUri.isDefined && !part.storage.locationUri.get.isAbsolute) { + val partPath = new Path(new Path(tbl.location), new Path(part.storage.locationUri.get)) + val qualifiedPartPath = makeQualifiedPath(CatalogUtils.stringToURI(partPath.toString)) + part.copy(storage = part.storage.copy(locationUri = Some(qualifiedPartPath))) + } else part + } + } // ---------------------------------------------------------------------------- // Functions // ---------------------------------------------------------------------------- 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 index bbe27831f01df..3e83c1dcb4758 100644 --- 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.csv +import org.apache.commons.lang3.StringUtils + object CSVExprUtils { /** * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). @@ -79,4 +81,48 @@ object CSVExprUtils { throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") } } + + /** + * Helper method that converts string representation of a character sequence to actual + * delimiter characters. The input is processed in "chunks", and each chunk is converted + * by calling [[CSVExprUtils.toChar()]]. A chunk is either: + *
    + *
  • a backslash followed by another character
  • + *
  • a non-backslash character by itself
  • + *
+ * , in that order of precedence. The result of the converting all chunks is returned as + * a [[String]]. + * + *

Examples: + *
  • `\t` will result in a single tab character as the separator (same as before) + *
  • `|||` will result in a sequence of three pipe characters as the separator + *
  • `\\` will result in a single backslash as the separator (same as before) + *
  • `\.` will result in an error (since a dot is not a character that needs escaped) + *
  • `\\.` will result in a backslash, then dot, as the separator character sequence + *
  • `.\t.` will result in a dot, then tab, then dot as the separator character sequence + *
  • + *
+ * + * @param str the string representing the sequence of separator characters + * @return a [[String]] representing the multi-character delimiter + * @throws IllegalArgumentException if any of the individual input chunks are illegal + */ + def toDelimiterStr(str: String): String = { + var idx = 0 + + var delimiter = "" + + while (idx < str.length()) { + // if the current character is a backslash, check it plus the next char + // in order to use existing escape logic + val readAhead = if (str(idx) == '\\') 2 else 1 + // get the chunk of 1 or 2 input characters to convert to a single delimiter char + val chunk = StringUtils.substring(str, idx, idx + readAhead) + delimiter += toChar(chunk) + // advance the counter by the length of input chunk processed + idx += chunk.length() + } + + delimiter.mkString("") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 8a91c71f1f145..5e40d74e54f11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -95,7 +95,7 @@ class CSVOptions( } } - val delimiter = CSVExprUtils.toChar( + val delimiter = CSVExprUtils.toDelimiterStr( parameters.getOrElse("sep", parameters.getOrElse("delimiter", ","))) val parseMode: ParseMode = parameters.get("mode").map(ParseMode.fromString).getOrElse(PermissiveMode) 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 d37d81753f0b8..c1cef013eab1c 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 @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.types._ /** 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 118f261de775d..862b2bb515a19 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 @@ -23,7 +23,7 @@ import java.util.Locale import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{InternalRow, WalkedTypePath} +import org.apache.spark.sql.catalyst.InternalRow 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._ @@ -32,7 +32,7 @@ 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.UTF8StringBuilder -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper} object Cast { @@ -165,6 +165,7 @@ object Cast { */ def canANSIStoreAssign(from: DataType, to: DataType): Boolean = (from, to) match { case _ if from == to => true + case (NullType, _) => true case (_: NumericType, _: NumericType) => true case (_: AtomicType, StringType) => true case (_: CalendarIntervalType, StringType) => true @@ -243,23 +244,11 @@ object Cast { } } -/** - * Cast the child expression to the target data type. - * - * When cast from/to timezone related types, we need timeZoneId, which will be resolved with - * session local timezone by an analyzer [[ResolveTimeZone]]. - */ -@ExpressionDescription( - usage = "_FUNC_(expr AS type) - Casts the value `expr` to the target data type `type`.", - examples = """ - Examples: - > SELECT _FUNC_('10' as int); - 10 - """) -case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String] = None) - extends UnaryExpression with TimeZoneAwareExpression with NullIntolerant { +abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression with NullIntolerant { - def this(child: Expression, dataType: DataType) = this(child, dataType, None) + def child: Expression + + def dataType: DataType override def toString: String = s"cast($child as ${dataType.simpleString})" @@ -274,8 +263,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String override def nullable: Boolean = Cast.forceNullable(child.dataType, dataType) || child.nullable - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) + protected def ansiEnabled: Boolean // When this cast involves TimeZone, it's only resolved if the timeZoneId is set; // Otherwise behave like Expression.resolved. @@ -289,7 +277,6 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) - private val failOnIntegralTypeOverflow = SQLConf.get.ansiEnabled // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { @@ -479,7 +466,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // IntervalConverter private[this] def castToInterval(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => CalendarInterval.fromString(s.toString)) + buildCast[UTF8String](_, s => IntervalUtils.safeFromString(s.toString)) } // LongConverter @@ -493,7 +480,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String buildCast[Int](_, d => null) case TimestampType => buildCast[Long](_, t => timestampToLong(t)) - case x: NumericType if failOnIntegralTypeOverflow => + case x: NumericType if ansiEnabled => b => x.exactNumeric.asInstanceOf[Numeric[Any]].toLong(b) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) @@ -508,11 +495,11 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String buildCast[Boolean](_, b => if (b) 1 else 0) case DateType => buildCast[Int](_, d => null) - case TimestampType if failOnIntegralTypeOverflow => + case TimestampType if ansiEnabled => buildCast[Long](_, t => LongExactNumeric.toInt(timestampToLong(t))) case TimestampType => buildCast[Long](_, t => timestampToLong(t).toInt) - case x: NumericType if failOnIntegralTypeOverflow => + case x: NumericType if ansiEnabled => b => x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) @@ -531,7 +518,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String buildCast[Boolean](_, b => if (b) 1.toShort else 0.toShort) case DateType => buildCast[Int](_, d => null) - case TimestampType if failOnIntegralTypeOverflow => + case TimestampType if ansiEnabled => buildCast[Long](_, t => { val longValue = timestampToLong(t) if (longValue == longValue.toShort) { @@ -542,7 +529,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String }) case TimestampType => buildCast[Long](_, t => timestampToLong(t).toShort) - case x: NumericType if failOnIntegralTypeOverflow => + case x: NumericType if ansiEnabled => b => val intValue = try { x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) @@ -572,7 +559,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String buildCast[Boolean](_, b => if (b) 1.toByte else 0.toByte) case DateType => buildCast[Int](_, d => null) - case TimestampType if failOnIntegralTypeOverflow => + case TimestampType if ansiEnabled => buildCast[Long](_, t => { val longValue = timestampToLong(t) if (longValue == longValue.toByte) { @@ -583,7 +570,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String }) case TimestampType => buildCast[Long](_, t => timestampToLong(t).toByte) - case x: NumericType if failOnIntegralTypeOverflow => + case x: NumericType if ansiEnabled => b => val intValue = try { x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) @@ -600,8 +587,6 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } - private val nullOnOverflow = !SQLConf.get.ansiEnabled - /** * Change the precision / scale in a given decimal to those set in `decimalType` (if any), * modifying `value` in-place and returning it if successful. If an overflow occurs, it @@ -614,7 +599,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (value.changePrecision(decimalType.precision, decimalType.scale)) { value } else { - if (nullOnOverflow) { + if (!ansiEnabled) { null } else { throw new ArithmeticException(s"${value.toDebugString} cannot be represented as " + @@ -630,7 +615,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = value.toPrecision( - decimalType.precision, decimalType.scale, Decimal.ROUND_HALF_UP, nullOnOverflow) + decimalType.precision, decimalType.scale, Decimal.ROUND_HALF_UP, !ansiEnabled) private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { @@ -1095,7 +1080,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String |$evPrim = $d; """.stripMargin } else { - val overflowCode = if (nullOnOverflow) { + val overflowCode = if (!ansiEnabled) { s"$evNull = true;" } else { s""" @@ -1228,8 +1213,9 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToIntervalCode(from: DataType): CastFunction = from match { case StringType => + val util = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") (c, evPrim, evNull) => - code"""$evPrim = CalendarInterval.fromString($c.toString()); + code"""$evPrim = $util.safeFromString($c.toString()); if(${evPrim} == null) { ${evNull} = true; } @@ -1274,7 +1260,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castTimestampToIntegralTypeCode( ctx: CodegenContext, integralType: String): CastFunction = { - if (failOnIntegralTypeOverflow) { + if (ansiEnabled) { val longValue = ctx.freshName("longValue") (c, evPrim, evNull) => code""" @@ -1293,7 +1279,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castDecimalToIntegralTypeCode( ctx: CodegenContext, integralType: String): CastFunction = { - if (failOnIntegralTypeOverflow) { + if (ansiEnabled) { (c, evPrim, evNull) => code"$evPrim = $c.roundTo${integralType.capitalize}();" } else { (c, evPrim, evNull) => code"$evPrim = $c.to${integralType.capitalize}();" @@ -1301,7 +1287,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String } private[this] def castIntegralTypeToIntegralTypeExactCode(integralType: String): CastFunction = { - assert(failOnIntegralTypeOverflow) + assert(ansiEnabled) (c, evPrim, evNull) => code""" if ($c == ($integralType) $c) { @@ -1329,7 +1315,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castFractionToIntegralTypeCode( fractionType: String, integralType: String): CastFunction = { - assert(failOnIntegralTypeOverflow) + assert(ansiEnabled) val (min, max) = lowerAndUpperBound(fractionType, integralType) val mathClass = classOf[Math].getName // When casting floating values to integral types, Spark uses the method `Numeric.toInt` @@ -1366,11 +1352,11 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String (c, evPrim, evNull) => code"$evNull = true;" case TimestampType => castTimestampToIntegralTypeCode(ctx, "byte") case DecimalType() => castDecimalToIntegralTypeCode(ctx, "byte") - case _: ShortType | _: IntegerType | _: LongType if failOnIntegralTypeOverflow => + case _: ShortType | _: IntegerType | _: LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("byte") - case _: FloatType if failOnIntegralTypeOverflow => + case _: FloatType if ansiEnabled => castFractionToIntegralTypeCode("float", "byte") - case _: DoubleType if failOnIntegralTypeOverflow => + case _: DoubleType if ansiEnabled => castFractionToIntegralTypeCode("double", "byte") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (byte) $c;" @@ -1397,11 +1383,11 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String (c, evPrim, evNull) => code"$evNull = true;" case TimestampType => castTimestampToIntegralTypeCode(ctx, "short") case DecimalType() => castDecimalToIntegralTypeCode(ctx, "short") - case _: IntegerType | _: LongType if failOnIntegralTypeOverflow => + case _: IntegerType | _: LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("short") - case _: FloatType if failOnIntegralTypeOverflow => + case _: FloatType if ansiEnabled => castFractionToIntegralTypeCode("float", "short") - case _: DoubleType if failOnIntegralTypeOverflow => + case _: DoubleType if ansiEnabled => castFractionToIntegralTypeCode("double", "short") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (short) $c;" @@ -1426,10 +1412,10 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String (c, evPrim, evNull) => code"$evNull = true;" case TimestampType => castTimestampToIntegralTypeCode(ctx, "int") case DecimalType() => castDecimalToIntegralTypeCode(ctx, "int") - case _: LongType if failOnIntegralTypeOverflow => castIntegralTypeToIntegralTypeExactCode("int") - case _: FloatType if failOnIntegralTypeOverflow => + case _: LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("int") + case _: FloatType if ansiEnabled => castFractionToIntegralTypeCode("float", "int") - case _: DoubleType if failOnIntegralTypeOverflow => + case _: DoubleType if ansiEnabled => castFractionToIntegralTypeCode("double", "int") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (int) $c;" @@ -1456,9 +1442,9 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case TimestampType => (c, evPrim, evNull) => code"$evPrim = (long) ${timestampToLongCode(c)};" case DecimalType() => castDecimalToIntegralTypeCode(ctx, "long") - case _: FloatType if failOnIntegralTypeOverflow => + case _: FloatType if ansiEnabled => castFractionToIntegralTypeCode("float", "long") - case _: DoubleType if failOnIntegralTypeOverflow => + case _: DoubleType if ansiEnabled => castFractionToIntegralTypeCode("double", "long") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (long) $c;" @@ -1647,6 +1633,43 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String } } +/** + * Cast the child expression to the target data type. + * + * When cast from/to timezone related types, we need timeZoneId, which will be resolved with + * session local timezone by an analyzer [[ResolveTimeZone]]. + */ +@ExpressionDescription( + usage = "_FUNC_(expr AS type) - Casts the value `expr` to the target data type `type`.", + examples = """ + Examples: + > SELECT _FUNC_('10' as int); + 10 + """) +case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String] = None) + extends CastBase { + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override protected val ansiEnabled: Boolean = SQLConf.get.ansiEnabled +} + +/** + * Cast the child expression to the target data type as per ANSI SQL standard. + * A runtime exception will be thrown on casting failure such as converting an out-of-range value + * to an integral type. + * + * When cast from/to timezone related types, we need timeZoneId, which will be resolved with + * session local timezone by an analyzer [[ResolveTimeZone]]. + */ +case class AnsiCast(child: Expression, dataType: DataType, timeZoneId: Option[String] = None) + extends CastBase { + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override protected val ansiEnabled: Boolean = true +} + /** * Cast the child expression to the target data type, but will throw error if the cast might * truncate, e.g. long -> int, timestamp -> data. 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 72ff9361d8f75..a32052ce121df 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.mutable +import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.expressions.objects.LambdaVariable @@ -72,7 +73,10 @@ class EquivalentExpressions { val skip = expr.isInstanceOf[LeafExpression] || // `LambdaVariable` is usually used as a loop variable, which can't be evaluated ahead of the // loop. So we can't evaluate sub-expressions containing `LambdaVariable` at the beginning. - expr.find(_.isInstanceOf[LambdaVariable]).isDefined + expr.find(_.isInstanceOf[LambdaVariable]).isDefined || + // `PlanExpression` wraps query plan. To compare query plans of `PlanExpression` on executor, + // can cause error like NPE. + (expr.isInstanceOf[PlanExpression[_]] && TaskContext.get != null) // There are some special expressions that we should not recurse into all of its children. // 1. CodegenFallback: it's children will not be used to generate code (call eval() instead) 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 eaaf94baac216..300f075d32763 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 @@ -127,12 +127,6 @@ object UnsafeProjection InterpretedUnsafeProjection.createProjection(in) } - protected def toUnsafeExprs(exprs: Seq[Expression]): Seq[Expression] = { - exprs.map(_ transform { - case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) - }) - } - /** * Returns an UnsafeProjection for given StructType. * @@ -153,7 +147,7 @@ object UnsafeProjection * Returns an UnsafeProjection for given sequence of bound Expressions. */ def create(exprs: Seq[Expression]): UnsafeProjection = { - createObject(toUnsafeExprs(exprs)) + createObject(exprs) } def create(expr: Expression): UnsafeProjection = create(Seq(expr)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala index 75feaf670c84a..159f90995c1fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala @@ -221,7 +221,7 @@ final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGen override protected def genericGet(i: Int): Any = values(i).boxed - override def update(ordinal: Int, value: Any) { + override def update(ordinal: Int, value: Any): Unit = { if (value == null) { setNullAt(ordinal) } else { 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 9aae678deb4bc..b9ec933f31493 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 @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( timeColumn: Expression, @@ -102,7 +102,7 @@ object TimeWindow { * precision. */ private def getIntervalInMicroSeconds(interval: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) + val cal = IntervalUtils.fromString(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 ea0ed2e8fa11b..59481ce049165 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 @@ -63,7 +63,7 @@ import org.apache.spark.sql.types._ > SELECT _FUNC_(10.0, array(0.5, 0.4, 0.1), 100); [10.0,10.0,10.0] > SELECT _FUNC_(10.0, 0.5, 100); - 10.0 + 10 """, since = "2.1.0") case class ApproximatePercentile( 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 1870c58c548c9..8ce8dfa19c017 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 @@ -223,7 +223,7 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { examples = """ Examples: > SELECT _FUNC_(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col); - 1.1135657469022013 + 1.1135657469022011 > SELECT _FUNC_(col) FROM VALUES (-1000), (-100), (10), (20) AS tab(col); -1.1135657469022011 """, @@ -245,9 +245,9 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) { examples = """ Examples: > SELECT _FUNC_(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col); - -0.7014368047529618 + -0.7014368047529627 > SELECT _FUNC_(col) FROM VALUES (1), (10), (100), (10), (1) as tab(col); - 0.19432323191698986 + 0.19432323191699075 """, since = "1.6.0") case class Kurtosis(child: Expression) extends CentralMomentAgg(child) { 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 index 4562fbcff5f3d..a8220ec641d6a 100644 --- 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 @@ -71,19 +71,3 @@ case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { 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/bitwiseAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala new file mode 100644 index 0000000000000..131fa2eb50555 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.expressions.{AttributeReference, BitwiseAnd, BitwiseOr, ExpectsInputTypes, Expression, ExpressionDescription, If, IsNull, Literal} +import org.apache.spark.sql.types.{AbstractDataType, DataType, IntegralType} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the bitwise AND of all non-null input values, or null if none.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); + 1 + """, + since = "3.0.0") +case class BitAndAgg(child: Expression) extends DeclarativeAggregate with ExpectsInputTypes { + + override def nodeName: String = "bit_and" + + override def children: Seq[Expression] = child :: Nil + + override def nullable: Boolean = true + + override def dataType: DataType = child.dataType + + override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) + + private lazy val bitAnd = AttributeReference("bit_and", child.dataType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = bitAnd :: Nil + + override lazy val initialValues: Seq[Literal] = Literal.create(null, dataType) :: Nil + + override lazy val updateExpressions: Seq[Expression] = + If(IsNull(bitAnd), + child, + If(IsNull(child), bitAnd, BitwiseAnd(bitAnd, child))) :: Nil + + override lazy val mergeExpressions: Seq[Expression] = + If(IsNull(bitAnd.left), + bitAnd.right, + If(IsNull(bitAnd.right), bitAnd.left, BitwiseAnd(bitAnd.left, bitAnd.right))) :: Nil + + override lazy val evaluateExpression: AttributeReference = bitAnd +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the bitwise OR of all non-null input values, or null if none.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); + 7 + """, + since = "3.0.0") +case class BitOrAgg(child: Expression) extends DeclarativeAggregate with ExpectsInputTypes { + + override def nodeName: String = "bit_or" + + override def children: Seq[Expression] = child :: Nil + + override def nullable: Boolean = true + + override def dataType: DataType = child.dataType + + override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) + + private lazy val bitOr = AttributeReference("bit_or", child.dataType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = bitOr :: Nil + + override lazy val initialValues: Seq[Literal] = Literal.create(null, dataType) :: Nil + + override lazy val updateExpressions: Seq[Expression] = + If(IsNull(bitOr), + child, + If(IsNull(child), bitOr, BitwiseOr(bitOr, child))) :: Nil + + override lazy val mergeExpressions: Seq[Expression] = + If(IsNull(bitOr.left), + bitOr.right, + If(IsNull(bitOr.right), bitOr.left, BitwiseOr(bitOr.left, bitOr.right))) :: Nil + + override lazy val evaluateExpression: AttributeReference = bitOr +} 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 d863f83686020..595d7db0bb493 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 @@ -546,7 +546,7 @@ abstract class TypedImperativeAggregate[T] extends ImperativeAggregate { private[this] val anyObjectType = ObjectType(classOf[AnyRef]) private def getBufferObject(bufferRow: InternalRow): T = { - bufferRow.get(mutableAggBufferOffset, anyObjectType).asInstanceOf[T] + getBufferObject(bufferRow, mutableAggBufferOffset) } final override lazy val aggBufferAttributes: Seq[AttributeReference] = { @@ -570,4 +570,21 @@ abstract class TypedImperativeAggregate[T] extends ImperativeAggregate { final def serializeAggregateBufferInPlace(buffer: InternalRow): Unit = { buffer(mutableAggBufferOffset) = serialize(getBufferObject(buffer)) } + + /** + * Merge an input buffer into the aggregation buffer, where both buffers contain the deserialized + * java object. This function is used by aggregating accumulators. + * + * @param buffer the aggregation buffer that is updated. + * @param inputBuffer the buffer that is merged into the aggregation buffer. + */ + final def mergeBuffersObjects(buffer: InternalRow, inputBuffer: InternalRow): Unit = { + val bufferObject = getBufferObject(buffer) + val inputObject = getBufferObject(inputBuffer, inputAggBufferOffset) + buffer(mutableAggBufferOffset) = merge(bufferObject, inputObject) + } + + private def getBufferObject(buffer: InternalRow, offset: Int): T = { + buffer.get(offset, anyObjectType).asInstanceOf[T] + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index e4276e33acbd2..f54a48370845a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -151,7 +151,7 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") // Name of the function for the exact version of this expression in [[Math]]. - // If the option "spark.sql.failOnIntegralTypeOverflow" is enabled and there is corresponding + // If the option "spark.sql.ansi.enabled" is enabled and there is corresponding // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. def exactMathMethod: Option[String] = None @@ -448,7 +448,7 @@ case class IntegralDivide(left: Expression, right: Expression) extends DivModLik usage = "expr1 _FUNC_ expr2 - Returns the remainder after `expr1`/`expr2`.", examples = """ Examples: - > SELECT 2 _FUNC_ 1.8; + > SELECT 2 % 1.8; 0.2 > SELECT MOD(2, 1.8); 0.2 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 c766bd8e56bb6..56f170cd77073 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 @@ -154,3 +154,35 @@ case class BitwiseNot(child: Expression) extends UnaryExpression with ExpectsInp override def sql: String = s"~${child.sql}" } + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the number of bits that are set in the argument expr as an" + + " unsigned 64-bit integer, or NULL if the argument is NULL.", + examples = """ + Examples: + > SELECT _FUNC_(0); + 0 + """) +case class BitwiseCount(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegralType, BooleanType)) + + override def dataType: DataType = IntegerType + + override def toString: String = s"bit_count($child)" + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = child.dataType match { + case BooleanType => defineCodeGen(ctx, ev, c => s"if ($c) 1 else 0") + case _ => defineCodeGen(ctx, ev, c => s"java.lang.Long.bitCount($c)") + } + + protected override def nullSafeEval(input: Any): Any = child.dataType match { + case BooleanType => if (input.asInstanceOf[Boolean]) 1 else 0 + case ByteType => java.lang.Long.bitCount(input.asInstanceOf[Byte]) + case ShortType => java.lang.Long.bitCount(input.asInstanceOf[Short]) + case IntegerType => java.lang.Long.bitCount(input.asInstanceOf[Int]) + case LongType => java.lang.Long.bitCount(input.asInstanceOf[Long]) + } + + override def sql: String = s"bit_count(${child.sql})" +} 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 5314821ea3a59..57b692ac16068 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 @@ -79,7 +79,7 @@ trait BinaryArrayExpressionWithImplicitCast extends BinaryExpression _FUNC_(expr) - Returns the size of an array or a map. The function returns -1 if its input is null and spark.sql.legacy.sizeOfNull is set to true. If spark.sql.legacy.sizeOfNull is set to false, the function returns null for null input. - By default, the spark.sql.legacy.sizeOfNull parameter is set to true. + By default, the spark.sql.legacy.sizeOfNull parameter is set to false. """, examples = """ Examples: @@ -88,7 +88,7 @@ trait BinaryArrayExpressionWithImplicitCast extends BinaryExpression > SELECT _FUNC_(map('a', 1, 'b', 2)); 2 > SELECT _FUNC_(NULL); - -1 + NULL """) case class Size(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -1428,7 +1428,7 @@ case class ArraysOverlap(left: Expression, right: Expression) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(x, start, length) - Subsets array x starting from index start (or starting from the end if start is negative) with the specified length.", + usage = "_FUNC_(x, start, length) - Subsets array x starting from index start (array indices start at 1, or starting from the end if start is negative) with the specified length.", examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, 4), 2, 2); 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 319a7fc87e59a..3f722e8537c36 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 @@ -295,9 +295,20 @@ object CreateStruct extends FunctionBuilder { } /** - * Common base class for both [[CreateNamedStruct]] and [[CreateNamedStructUnsafe]]. + * Creates a struct with the given field names and values + * + * @param children Seq(name1, val1, name2, val2, ...) */ -trait CreateNamedStructLike extends Expression { +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.", + examples = """ + Examples: + > SELECT _FUNC_("a", 1, "b", 2, "c", 3); + {"a":1,"b":2,"c":3} + """) +// scalastyle:on line.size.limit +case class CreateNamedStruct(children: Seq[Expression]) extends Expression { lazy val (nameExprs, valExprs) = children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip @@ -348,23 +359,6 @@ trait CreateNamedStructLike extends Expression { override def eval(input: InternalRow): Any = { InternalRow(valExprs.map(_.eval(input)): _*) } -} - -/** - * Creates a struct with the given field names and values - * - * @param children Seq(name1, val1, name2, val2, ...) - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.", - examples = """ - Examples: - > SELECT _FUNC_("a", 1, "b", 2, "c", 3); - {"a":1,"b":2,"c":3} - """) -// scalastyle:on line.size.limit -case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rowClass = classOf[GenericInternalRow].getName @@ -397,22 +391,6 @@ case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStruc override def prettyName: String = "named_struct" } -/** - * Creates a struct with the given field names and values. This is a variant that returns - * UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with - * this expression automatically at runtime. - * - * @param children Seq(name1, val1, name2, val2, ...) - */ -case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateNamedStructLike { - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) - ExprCode(code = eval.code, isNull = FalseLiteral, value = eval.value) - } - - override def prettyName: String = "named_struct_unsafe" -} - /** * Creates a map after splitting the input text into key/value pairs using delimiters */ @@ -422,9 +400,9 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateName examples = """ Examples: > SELECT _FUNC_('a:1,b:2,c:3', ',', ':'); - map("a":"1","b":"2","c":"3") + {"a":"1","b":"2","c":"3"} > SELECT _FUNC_('a'); - map("a":null) + {"a":null} """) // scalastyle:on line.size.limit case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: Expression) 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 index 65b10f36373d1..67c24f687af08 100644 --- 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 @@ -40,8 +40,8 @@ import org.apache.spark.unsafe.types.UTF8String 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')) + {"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") @@ -199,7 +199,7 @@ case class SchemaOfCsv( > 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" + 26/08/2015 """, since = "3.0.0") // scalastyle:on line.size.limit 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 0098226b3258c..90eb3598b233b 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,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, ZoneId} +import java.time.{DateTimeException, LocalDate, LocalDateTime, ZoneId} import java.time.temporal.IsoFields import java.util.{Locale, TimeZone} @@ -298,6 +298,30 @@ case class Second(child: Expression, timeZoneId: Option[String] = None) } } +case class SecondWithFraction(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(child: Expression) = this(child, None) + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + + // 2 digits for seconds, and 6 digits for the fractional part with microsecond precision. + override def dataType: DataType = DecimalType(8, 6) + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override protected def nullSafeEval(timestamp: Any): Any = { + DateTimeUtils.getSecondsWithFraction(timestamp.asInstanceOf[Long], timeZone) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceObj("timeZone", timeZone) + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getSecondsWithFraction($c, $tz)") + } +} + case class Milliseconds(child: Expression, timeZoneId: Option[String] = None) extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression { @@ -631,7 +655,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti examples = """ Examples: > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd'); - 1460041200 + 1460098800 """, since = "1.6.0") case class ToUnixTimestamp( @@ -842,7 +866,7 @@ abstract class UnixTime extends ToTimestamp { examples = """ Examples: > SELECT _FUNC_(0, 'yyyy-MM-dd HH:mm:ss'); - 1970-01-01 00:00:00 + 1969-12-31 16:00:00 """, since = "1.5.0") case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[String] = None) @@ -1097,7 +1121,7 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S usage = "_FUNC_(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 03:40:00.0'.", examples = """ Examples: - > SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul'); + > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); 2016-08-31 09:00:00 """, since = "1.5.0", @@ -1766,10 +1790,10 @@ case class MakeDate(year: Expression, month: Expression, day: Expression) > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887); 2014-12-28 06:30:45.887 > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET'); - 2014-12-28 10:30:45.887 - > SELECT _FUNC_(2019, 6, 30, 23, 59, 60) + 2014-12-27 21:30:45.887 + > SELECT _FUNC_(2019, 6, 30, 23, 59, 60); 2019-07-01 00:00:00 - > SELECT _FUNC_(2019, 13, 1, 10, 11, 12, 13); + > SELECT _FUNC_(2019, 13, 1, 10, 11, 12, 'PST'); NULL > SELECT _FUNC_(null, 7, 22, 15, 30, 0); NULL @@ -1997,7 +2021,7 @@ object DatePart { case "DOY" => DayOfYear(source) case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) - case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => SecondWithFraction(source) case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => Milliseconds(source) case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => @@ -2008,10 +2032,11 @@ object DatePart { } @ExpressionDescription( - usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp.", + usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp or interval source.", arguments = """ Arguments: - * field - selects which part of the source should be extracted. Supported string values are: + * field - selects which part of the source should be extracted. + Supported string values of `field` for dates and timestamps are: ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), "CENTURY", ("CENTURIES", "C", "CENT"), "DECADE", ("DECADES", "DEC", "DECS"), @@ -2031,7 +2056,21 @@ object DatePart { "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), "EPOCH"] - * source - a date (or timestamp) column from where `field` should be extracted + Supported string values of `field` for intervals are: + ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), + "CENTURY", ("CENTURIES", "C", "CENT"), + "DECADE", ("DECADES", "DEC", "DECS"), + "YEAR", ("Y", "YEARS", "YR", "YRS"), + "QUARTER", ("QTR"), + "MONTH", ("MON", "MONS", "MONTHS"), + "DAY", ("D", "DAYS"), + "HOUR", ("H", "HOURS", "HR", "HRS"), + "MINUTE", ("M", "MIN", "MINS", "MINUTES"), + "SECOND", ("S", "SEC", "SECONDS", "SECS"), + "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), + "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), + "EPOCH"] + * source - a date/timestamp or interval column from where `field` should be extracted """, examples = """ Examples: @@ -2041,6 +2080,12 @@ object DatePart { 33 > SELECT _FUNC_('doy', DATE'2019-08-12'); 224 + > SELECT _FUNC_('SECONDS', timestamp'2019-10-01 00:00:01.000001'); + 1.000001 + > SELECT _FUNC_('days', interval 1 year 10 months 5 days); + 5 + > SELECT _FUNC_('seconds', interval 5 hours 30 seconds 1 milliseconds 1 microseconds); + 30.001001 """, since = "3.0.0") case class DatePart(field: Expression, source: Expression, child: Expression) @@ -2056,9 +2101,16 @@ case class DatePart(field: Expression, source: Expression, child: Expression) Literal(null, DoubleType) } else { val fieldStr = fieldEval.asInstanceOf[UTF8String].toString - DatePart.parseExtractField(fieldStr, source, { - throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") - }) + val errMsg = s"Literals of type '$fieldStr' are currently not supported " + + s"for the ${source.dataType.catalogString} type." + if (source.dataType == CalendarIntervalType) { + ExtractIntervalPart.parseExtractField( + fieldStr, + source, + throw new AnalysisException(errMsg)) + } else { + DatePart.parseExtractField(fieldStr, source, throw new AnalysisException(errMsg)) + } } }) } @@ -2067,3 +2119,48 @@ case class DatePart(field: Expression, source: Expression, child: Expression) override def sql: String = s"$prettyName(${field.sql}, ${source.sql})" override def prettyName: String = "date_part" } + +/** + * Returns the interval from startTimestamp to endTimestamp in which the `months` field + * is set to 0 and the `microseconds` field is initialized to the microsecond difference + * between the given timestamps. + */ +case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expression) + extends BinaryExpression with ImplicitCastInputTypes { + + override def left: Expression = endTimestamp + override def right: Expression = startTimestamp + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, TimestampType) + override def dataType: DataType = CalendarIntervalType + + override def nullSafeEval(end: Any, start: Any): Any = { + new CalendarInterval(0, end.asInstanceOf[Long] - start.asInstanceOf[Long]) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (end, start) => + s"new org.apache.spark.unsafe.types.CalendarInterval(0, $end - $start)") + } +} + +/** + * Returns the interval from the `left` date (inclusive) to the `right` date (exclusive). + */ +case class SubtractDates(left: Expression, right: Expression) + extends BinaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType, DateType) + override def dataType: DataType = CalendarIntervalType + + override def nullSafeEval(leftDays: Any, rightDays: Any): Any = { + DateTimeUtils.subtractDates(leftDays.asInstanceOf[Int], rightDays.asInstanceOf[Int]) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (leftDays, rightDays) => { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + s"$dtu.subtractDates($leftDays, $rightDays)" + }) + } +} + 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 82a7d9825e30a..b0a23c62284d2 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 @@ -127,16 +127,16 @@ case class UserDefinedGenerator( * 3 NULL * }}} */ -// scalastyle:off line.size.limit +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( 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 + 1 2 + 3 NULL """) -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab case class Stack(children: Seq[Expression]) extends Generator { private lazy val numRows = children.head.eval().asInstanceOf[Int] @@ -375,16 +375,16 @@ case class Explode(child: Expression) extends ExplodeBase { * 1 20 * }}} */ -// scalastyle:off line.size.limit +// scalastyle:off line.size.limit line.contains.tab @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. 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)); - 0 10 - 1 20 + 0 10 + 1 20 """) -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab case class PosExplode(child: Expression) extends ExplodeBase { override val position = true } @@ -392,16 +392,16 @@ case class PosExplode(child: Expression) extends ExplodeBase { /** * Explodes an array of structs into a table. */ -// scalastyle:off line.size.limit +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( 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 + 1 a + 2 b """) -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab 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 221b97bdc7856..2da3f6c74b0e2 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,7 +38,7 @@ trait GroupingSet extends Expression with CodegenFallback { override def eval(input: InternalRow): Any = throw new UnsupportedOperationException } -// scalastyle:off line.size.limit +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( usage = """ _FUNC_([col1[, col2 ..]]) - create a multi-dimensional cube using the specified columns @@ -47,19 +47,19 @@ trait GroupingSet extends Expression with CodegenFallback { 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 + Bob 5 1 + Alice 2 1 + NULL NULL 2 + NULL 5 1 + Bob NULL 1 + Alice NULL 1 + NULL 2 1 """, since = "2.0.0") -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab case class Cube(groupByExprs: Seq[Expression]) extends GroupingSet {} -// scalastyle:off line.size.limit +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( usage = """ _FUNC_([col1[, col2 ..]]) - create a multi-dimensional rollup using the specified columns @@ -68,21 +68,21 @@ case class Cube(groupByExprs: Seq[Expression]) extends GroupingSet {} 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 + Bob 5 1 + Alice 2 1 + NULL NULL 2 + Bob NULL 1 + Alice NULL 1 """, since = "2.0.0") -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab 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 +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( usage = """ _FUNC_(col) - indicates whether a specified column in a GROUP BY is aggregated or @@ -91,12 +91,12 @@ case class Rollup(groupByExprs: Seq[Expression]) extends GroupingSet {} 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 + Bob 0 5 + Alice 0 2 + NULL 1 7 """, since = "2.0.0") -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab case class Grouping(child: Expression) extends Expression with Unevaluable { @transient override lazy val references: AttributeSet = @@ -111,7 +111,7 @@ 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 +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( usage = """ _FUNC_([col1[, col2 ..]]) - returns the level of grouping, equals to @@ -120,20 +120,20 @@ case class Grouping(child: Expression) extends Expression with Unevaluable { 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 + NULL 2 5 180.0 + Alice 0 2 165.0 + NULL 3 7 172.5 + NULL 2 2 165.0 + Bob 1 5 180.0 + Alice 1 2 165.0 + Bob 0 5 180.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 +// scalastyle:on line.size.limit line.contains.tab case class GroupingID(groupByExprs: Seq[Expression]) extends Expression with Unevaluable { @transient override lazy val references: AttributeSet = 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 ed26bb375de25..317ebb62c07ec 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 @@ -344,8 +344,13 @@ case class MapFilter( Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1); [1,3] + > SELECT _FUNC_(array(0, 2, 3), (x, i) -> x > i); + [2,3] """, - since = "2.4.0") + since = "2.4.0", + note = """ + The inner function may use the index argument since 3.0.0. + """) case class ArrayFilter( argument: Expression, function: Expression) @@ -357,10 +362,19 @@ case class ArrayFilter( override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayFilter = { val ArrayType(elementType, containsNull) = argument.dataType - copy(function = f(function, (elementType, containsNull) :: Nil)) + function match { + case LambdaFunction(_, arguments, _) if arguments.size == 2 => + copy(function = f(function, (elementType, containsNull) :: (IntegerType, false) :: Nil)) + case _ => + copy(function = f(function, (elementType, containsNull) :: Nil)) + } } - @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function + @transient lazy val (elementVar, indexVar) = { + val LambdaFunction(_, (elementVar: NamedLambdaVariable) +: tail, _) = function + val indexVar = tail.headOption.map(_.asInstanceOf[NamedLambdaVariable]) + (elementVar, indexVar) + } override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { val arr = argumentValue.asInstanceOf[ArrayData] @@ -369,6 +383,9 @@ case class ArrayFilter( var i = 0 while (i < arr.numElements) { elementVar.value.set(arr.get(i, elementVar.dataType)) + if (indexVar.isDefined) { + indexVar.get.value.set(i) + } if (f.eval(inputRow).asInstanceOf[Boolean]) { buffer += elementVar.value.get } @@ -463,7 +480,7 @@ case class ArrayExists( > SELECT _FUNC_(array(1, null, 3), x -> x % 2 == 0); false > SELECT _FUNC_(array(2, null, 8), x -> x % 2 == 0); - null + NULL """, since = "3.0.0") case class ArrayForAll( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala new file mode 100644 index 0000000000000..08360c75a474b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -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.spark.sql.catalyst.expressions + +import java.util.Locale + +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.util.IntervalUtils +import org.apache.spark.sql.catalyst.util.IntervalUtils._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +abstract class ExtractIntervalPart( + child: Expression, + val dataType: DataType, + func: CalendarInterval => Any, + funcName: String) + extends UnaryExpression with ExpectsInputTypes with Serializable { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override protected def nullSafeEval(interval: Any): Any = { + func(interval.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.$funcName($c)") + } +} + +case class ExtractIntervalMillenniums(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getMillenniums, "getMillenniums") + +case class ExtractIntervalCenturies(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getCenturies, "getCenturies") + +case class ExtractIntervalDecades(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getDecades, "getDecades") + +case class ExtractIntervalYears(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getYears, "getYears") + +case class ExtractIntervalQuarters(child: Expression) + extends ExtractIntervalPart(child, ByteType, getQuarters, "getQuarters") + +case class ExtractIntervalMonths(child: Expression) + extends ExtractIntervalPart(child, ByteType, getMonths, "getMonths") + +case class ExtractIntervalDays(child: Expression) + extends ExtractIntervalPart(child, LongType, getDays, "getDays") + +case class ExtractIntervalHours(child: Expression) + extends ExtractIntervalPart(child, ByteType, getHours, "getHours") + +case class ExtractIntervalMinutes(child: Expression) + extends ExtractIntervalPart(child, ByteType, getMinutes, "getMinutes") + +case class ExtractIntervalSeconds(child: Expression) + extends ExtractIntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") + +case class ExtractIntervalMilliseconds(child: Expression) + extends ExtractIntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") + +case class ExtractIntervalMicroseconds(child: Expression) + extends ExtractIntervalPart(child, LongType, getMicroseconds, "getMicroseconds") + +// Number of seconds in 10000 years is 315576000001 (30 days per one month) +// which is 12 digits + 6 digits for the fractional part of seconds. +case class ExtractIntervalEpoch(child: Expression) + extends ExtractIntervalPart(child, DecimalType(18, 6), getEpoch, "getEpoch") + +object ExtractIntervalPart { + + def parseExtractField( + extractField: String, + source: Expression, + errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => ExtractIntervalMillenniums(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => ExtractIntervalCenturies(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => ExtractIntervalDecades(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => ExtractIntervalYears(source) + case "QUARTER" | "QTR" => ExtractIntervalQuarters(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => ExtractIntervalMonths(source) + case "DAY" | "D" | "DAYS" => ExtractIntervalDays(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => ExtractIntervalHours(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => ExtractIntervalMinutes(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => ExtractIntervalSeconds(source) + case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => + ExtractIntervalMilliseconds(source) + case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => + ExtractIntervalMicroseconds(source) + case "EPOCH" => ExtractIntervalEpoch(source) + case _ => errorHandleFunc + } +} 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 655e44e4e4919..a13a6836c6be6 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 @@ -331,15 +331,15 @@ case class GetJsonObject(json: Expression, path: Expression) } } -// scalastyle:off line.size.limit +// scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( usage = "_FUNC_(jsonStr, p1, p2, ..., pn) - Returns a tuple like the function get_json_object, but it takes multiple names. All the input parameters and output column types are string.", examples = """ Examples: > SELECT _FUNC_('{"a":1, "b":2}', 'a', 'b'); - 1 2 + 1 2 """) -// scalastyle:on line.size.limit +// scalastyle:on line.size.limit line.contains.tab case class JsonTuple(children: Seq[Expression]) extends Generator with CodegenFallback { @@ -502,9 +502,9 @@ case class JsonTuple(children: Seq[Expression]) examples = """ Examples: > SELECT _FUNC_('{"a":1, "b":0.8}', 'a INT, b DOUBLE'); - {"a":1, "b":0.8} + {"a":1,"b":0.8} > SELECT _FUNC_('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); - {"time":"2015-08-26 00:00:00.0"} + {"time":2015-08-26 00:00:00.0} """, since = "2.2.0") // scalastyle:on line.size.limit 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 4793b5942a79e..f3c129934314c 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 @@ -223,6 +223,26 @@ object NonNullLiteral { } } +/** + * Extractor for retrieving Float literals. + */ +object FloatLiteral { + def unapply(a: Any): Option[Float] = a match { + case Literal(a: Float, FloatType) => Some(a) + case _ => None + } +} + +/** + * Extractor for retrieving Double literals. + */ +object DoubleLiteral { + def unapply(a: Any): Option[Double] = a match { + case Literal(a: Double, DoubleType) => Some(a) + case _ => None + } +} + /** * Extractor for retrieving Int literals. */ @@ -233,6 +253,16 @@ object IntegerLiteral { } } +/** + * Extractor for retrieving String literals. + */ +object StringLiteral { + def unapply(a: Any): Option[String] = a match { + case Literal(s: UTF8String, StringType) => Some(s.toString) + case _ => None + } +} + /** * Extractor for and other utility methods for decimal literals. */ @@ -360,7 +390,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case _ if v.isNaN => "'NaN'" case Float.PositiveInfinity => "'Infinity'" case Float.NegativeInfinity => "'-Infinity'" - case _ => v + case _ => s"'$v'" } s"CAST($castedValue AS ${FloatType.sql})" case (v: Double, DoubleType) => 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 b9f089ec056c2..d5b959b91c23d 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 @@ -1291,7 +1291,7 @@ abstract class RoundBase(child: Expression, scale: Expression, examples = """ Examples: > SELECT _FUNC_(2.5, 0); - 3.0 + 3 """) // scalastyle:on line.size.limit case class Round(child: Expression, scale: Expression) @@ -1311,7 +1311,7 @@ case class Round(child: Expression, scale: Expression) examples = """ Examples: > SELECT _FUNC_(2.5, 0); - 2.0 + 2 """) // scalastyle:on line.size.limit case class BRound(child: Expression, scale: Expression) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala new file mode 100644 index 0000000000000..0e87707d01e47 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.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.catalyst.expressions.postgreSQL + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, JavaCode} +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils +import org.apache.spark.sql.types.{BooleanType, DataType, StringType} +import org.apache.spark.unsafe.types.UTF8String + +case class PostgreCastStringToBoolean(child: Expression) + extends UnaryExpression with NullIntolerant { + + override def checkInputDataTypes(): TypeCheckResult = { + if (child.dataType == StringType) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure( + s"The expression ${getClass.getSimpleName} only accepts string input data type") + } + } + + override def nullSafeEval(input: Any): Any = { + val s = input.asInstanceOf[UTF8String].trim().toLowerCase() + if (StringUtils.isTrueString(s)) { + true + } else if (StringUtils.isFalseString(s)) { + false + } else { + null + } + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" + val eval = child.genCode(ctx) + val javaType = JavaCode.javaType(dataType) + val preprocessedString = ctx.freshName("preprocessedString") + val castCode = + code""" + boolean ${ev.isNull} = ${eval.isNull}; + $javaType ${ev.value} = false; + if (!${eval.isNull}) { + UTF8String $preprocessedString = ${eval.value}.trim().toLowerCase(); + if ($stringUtils.isTrueString($preprocessedString)) { + ${ev.value} = true; + } else if ($stringUtils.isFalseString($preprocessedString)) { + ${ev.value} = false; + } else { + ${ev.isNull} = true; + } + } + """ + ev.copy(code = eval.code + castCode) + } + + override def dataType: DataType = BooleanType + + override def nullable: Boolean = true + + override def toString: String = s"PostgreCastStringToBoolean($child as ${dataType.simpleString})" + + override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" +} 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 9229ef2039fed..f52e903e75094 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 @@ -65,7 +65,7 @@ abstract class StringRegexExpression extends BinaryExpression override def sql: String = s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}" } - +// scalastyle:off line.contains.tab /** * Simple RegEx pattern matching function */ @@ -96,13 +96,20 @@ abstract class StringRegexExpression extends BinaryExpression """, examples = """ Examples: - > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%' + > SET spark.sql.parser.escapedStringLiterals=true; + spark.sql.parser.escapedStringLiterals true + > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%'; + true + > SET spark.sql.parser.escapedStringLiterals=false; + spark.sql.parser.escapedStringLiterals false + > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '\%SystemDrive\%\\\\Users%'; true """, note = """ Use RLIKE to match with standard regular expressions. """, since = "1.0.0") +// scalastyle:on line.contains.tab case class Like(left: Expression, right: Expression) extends StringRegexExpression { override def escape(v: String): String = StringUtils.escapeLikeRegex(v) @@ -153,6 +160,7 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi } } +// scalastyle:off line.contains.tab @ExpressionDescription( usage = "str _FUNC_ regexp - Returns true if `str` matches `regexp`, or false otherwise.", arguments = """ @@ -170,18 +178,20 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi """, examples = """ Examples: - When spark.sql.parser.escapedStringLiterals is disabled (default). - > SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\\Users.*' + > SET spark.sql.parser.escapedStringLiterals=true; + spark.sql.parser.escapedStringLiterals true + > SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\\Users.*'; true - - When spark.sql.parser.escapedStringLiterals is enabled. - > SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\Users.*' + > SET spark.sql.parser.escapedStringLiterals=false; + spark.sql.parser.escapedStringLiterals false + > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '%SystemDrive%\\\\Users.*'; true """, note = """ Use LIKE to match with simple string pattern. """, since = "1.0.0") +// scalastyle:on line.contains.tab case class RLike(left: Expression, right: Expression) extends StringRegexExpression { override def escape(v: String): String = v 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 e4847e9cec3f0..211ae3f02a0d8 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 @@ -847,7 +847,7 @@ object StringTrimLeft { usage = """ _FUNC_(str) - Removes the leading space characters from `str`. - _FUNC_(trimStr, str) - Removes the leading string contains the characters from the trim string + _FUNC_(str, trimStr) - Removes the leading string contains the characters from the trim string """, arguments = """ Arguments: @@ -858,7 +858,7 @@ object StringTrimLeft { Examples: > SELECT _FUNC_(' SparkSQL '); SparkSQL - > SELECT _FUNC_('Sp', 'SSparkSQLS'); + > SELECT _FUNC_('SparkSQLS', 'Sp'); arkSQLS """, since = "1.5.0") @@ -949,7 +949,7 @@ object StringTrimRight { usage = """ _FUNC_(str) - Removes the trailing space characters from `str`. - _FUNC_(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the `str` + _FUNC_(str, trimStr) - Removes the trailing string which contains the characters from the trim string from the `str` """, arguments = """ Arguments: @@ -960,7 +960,7 @@ object StringTrimRight { Examples: > SELECT _FUNC_(' SparkSQL '); SparkSQL - > SELECT _FUNC_('LQSa', 'SSparkSQLS'); + > SELECT _FUNC_('SSparkSQLS', 'SQLS'); SSpark """, since = "1.5.0") @@ -1299,11 +1299,11 @@ object ParseUrl { usage = "_FUNC_(url, partToExtract[, key]) - Extracts a part from a URL.", examples = """ Examples: - > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'HOST') + > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'HOST'); spark.apache.org - > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY') + > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY'); query=1 - > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY', 'query') + > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY', 'query'); 1 """, since = "2.0.0") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala index aacf1a44e2ad0..073b45af51caf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala @@ -194,7 +194,7 @@ case class XPathString(xml: Expression, path: Expression) extends XPathExtract { examples = """ Examples: > SELECT _FUNC_('b1b2b3c1c2','a/b/text()'); - ['b1','b2','b3'] + ["b1","b2","b3"] """) // scalastyle:on line.size.limit case class XPathList(xml: Expression, path: Expression) extends XPathExtract { 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 dc26a28c74f11..4952540f1132d 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 @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.{JsonFactory, JsonParser} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf /** * Options for parsing JSON data into Spark SQL rows. @@ -76,6 +77,10 @@ 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) + // Whether to ignore null fields during json generating + val ignoreNullFields = parameters.get("ignoreNullFields").map(_.toBoolean) + .getOrElse(SQLConf.get.jsonGeneratorIgnoreNullFields) + // A language tag in IETF BCP 47 format val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) 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 3ee7e484690d5..aaf2ecf7923ce 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 @@ -184,6 +184,9 @@ private[sql] class JacksonGenerator( if (!row.isNullAt(i)) { gen.writeFieldName(field.name) fieldWriters(i).apply(row, i) + } else if (!options.ignoreNullFields) { + gen.writeFieldName(field.name) + gen.writeNull() } i += 1 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala index db7d6d3254bd2..1743565ccb6c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule /** - * Simplify redundant [[CreateNamedStructLike]], [[CreateArray]] and [[CreateMap]] expressions. + * Simplify redundant [[CreateNamedStruct]], [[CreateArray]] and [[CreateMap]] expressions. */ object SimplifyExtractValueOps extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -37,8 +37,8 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] { case a: Aggregate => a case p => p.transformExpressionsUp { // Remove redundant field extraction. - case GetStructField(createNamedStructLike: CreateNamedStructLike, ordinal, _) => - createNamedStructLike.valExprs(ordinal) + case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) => + createNamedStruct.valExprs(ordinal) // Remove redundant array indexing. case GetArrayStructFields(CreateArray(elems), field, ordinal, _, _) => 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 index b036092cf1fcc..ea01d9e63eef7 100644 --- 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 @@ -17,7 +17,7 @@ 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.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, 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} @@ -114,9 +114,6 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case CreateNamedStruct(children) => CreateNamedStruct(children.map(normalize)) - case CreateNamedStructUnsafe(children) => - CreateNamedStructUnsafe(children.map(normalize)) - case CreateArray(children) => CreateArray(children.map(normalize)) 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 b992ab130bbcc..eab4c3efe4f7f 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ 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.connector.catalog.CatalogManager import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -35,7 +36,7 @@ import org.apache.spark.util.Utils * Abstract class all optimizers should inherit of, contains the standard batches (extending * Optimizers can override this. */ -abstract class Optimizer(sessionCatalog: SessionCatalog) +abstract class Optimizer(catalogManager: CatalogManager) extends RuleExecutor[LogicalPlan] { // Check for structural integrity of the plan in test mode. @@ -129,7 +130,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) EliminateView, ReplaceExpressions, ComputeCurrentTime, - GetCurrentDatabase(sessionCatalog), + GetCurrentDatabase(catalogManager), RewriteDistinctAggregates, ReplaceDeduplicateWithAggregate) :: ////////////////////////////////////////////////////////////////////////////////////////// @@ -212,7 +213,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) EliminateView.ruleName :: ReplaceExpressions.ruleName :: ComputeCurrentTime.ruleName :: - GetCurrentDatabase(sessionCatalog).ruleName :: + GetCurrentDatabase(catalogManager).ruleName :: RewriteDistinctAggregates.ruleName :: ReplaceDeduplicateWithAggregate.ruleName :: ReplaceIntersectWithSemiJoin.ruleName :: @@ -318,10 +319,10 @@ object EliminateDistinct extends Rule[LogicalPlan] { object SimpleTestOptimizer extends SimpleTestOptimizer class SimpleTestOptimizer extends Optimizer( - new SessionCatalog( - new InMemoryCatalog, - EmptyFunctionRegistry, - new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true))) + new CatalogManager( + new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true), + FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, new SQLConf()))) /** * Remove redundant aliases from a query plan. A redundant alias is an alias that does not change 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 0a6737ba42118..36ad796c08a38 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 @@ -227,8 +227,8 @@ object OptimizeIn extends Rule[LogicalPlan] { if (newList.length == 1 // TODO: `EqualTo` for structural types are not working. Until SPARK-24443 is addressed, // TODO: we exclude them in this rule. - && !v.isInstanceOf[CreateNamedStructLike] - && !newList.head.isInstanceOf[CreateNamedStructLike]) { + && !v.isInstanceOf[CreateNamedStruct] + && !newList.head.isInstanceOf[CreateNamedStruct]) { EqualTo(v, newList.head) } else if (newList.length > SQLConf.get.optimizerInSetConversionThreshold) { val hSet = newList.map(e => e.eval(EmptyRow)) 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 69ba76827c781..70277526cba8c 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 @@ -21,12 +21,12 @@ 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 +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.types._ @@ -47,7 +47,6 @@ 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) } @@ -78,11 +77,14 @@ object ComputeCurrentTime extends Rule[LogicalPlan] { /** Replaces the expression of CurrentDatabase with the current database name. */ -case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[LogicalPlan] { +case class GetCurrentDatabase(catalogManager: CatalogManager) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + val currentNamespace = catalogManager.currentNamespace.quoted + plan transformAllExpressions { case CurrentDatabase() => - Literal.create(sessionCatalog.getCurrentDatabase, StringType) + Literal.create(currentNamespace, StringType) } } } 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 74c485e091cd8..b030227b48814 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 @@ -36,8 +36,8 @@ 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, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -101,6 +101,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList))) } + override def visitSingleInterval(ctx: SingleIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val units = ctx.intervalUnit().asScala.map { + u => normalizeInternalUnit(u.getText.toLowerCase(Locale.ROOT)) + }.toArray + val values = ctx.intervalValue().asScala.map(getIntervalValue).toArray + try { + CalendarInterval.fromUnitStrings(units, values) + } catch { + case i: IllegalArgumentException => + val e = new ParseException(i.getMessage, ctx) + e.setStackTrace(i.getStackTrace) + throw e + } + } + } + /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -361,6 +378,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging DeleteFromStatement(tableId, tableAlias, predicate) } + override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { + val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) + val tableAlias = if (ctx.tableAlias() != null) { + val ident = ctx.tableAlias().strictIdentifier() + // We do not allow columns aliases after table alias. + if (ctx.tableAlias().identifierList() != null) { + throw new ParseException("Columns aliases is not allowed in UPDATE.", + ctx.tableAlias().identifierList()) + } + if (ident != null) Some(ident.getText) else None + } else { + None + } + val (attrs, values) = ctx.setClause().assign().asScala.map { + kv => visitMultipartIdentifier(kv.key) -> expression(kv.value) + }.unzip + val predicate = if (ctx.whereClause() != null) { + Some(expression(ctx.whereClause().booleanExpression())) + } else { + None + } + + UpdateTableStatement( + tableId, + tableAlias, + attrs, + values, + predicate) + } + /** * Create a partition specification map. */ @@ -1740,7 +1787,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => - Literal(CalendarInterval.fromString(value), CalendarIntervalType) + val interval = try { + IntervalUtils.fromString(value) + } catch { + case e: IllegalArgumentException => + val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx) + ex.setStackTrace(e.getStackTrace) + throw ex + } + Literal(interval, CalendarIntervalType) case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) @@ -1893,15 +1948,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) { import ctx._ - val s = value.getText + val s = getIntervalValue(value) try { val unitText = unit.getText.toLowerCase(Locale.ROOT) val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match { - case (u, None) if u.endsWith("s") => - // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... - CalendarInterval.fromSingleUnitString(u.substring(0, u.length - 1), s) case (u, None) => - CalendarInterval.fromSingleUnitString(u, s) + CalendarInterval.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s)) case ("year", Some("month")) => CalendarInterval.fromYearMonthString(s) case ("day", Some("hour")) => @@ -1930,6 +1982,19 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + private def getIntervalValue(value: IntervalValueContext): String = { + if (value.STRING() != null) { + string(value.STRING()) + } else { + value.getText + } + } + + // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... + private def normalizeInternalUnit(s: String): String = { + if (s.endsWith("s")) s.substring(0, s.length - 1) else s + } + /* ******************************************************************************************** * DataType parsing * ******************************************************************************************** */ @@ -2270,10 +2335,54 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create a [[CreateNamespaceStatement]] command. + * + * For example: + * {{{ + * CREATE NAMESPACE [IF NOT EXISTS] ns1.ns2.ns3 + * create_namespace_clauses; + * + * create_namespace_clauses (order insensitive): + * [COMMENT namespace_comment] + * [LOCATION path] + * [WITH PROPERTIES (key1=val1, key2=val2, ...)] + * }}} + */ + override def visitCreateNamespace(ctx: CreateNamespaceContext): LogicalPlan = withOrigin(ctx) { + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + checkDuplicateClauses(ctx.PROPERTIES, "WITH PROPERTIES", ctx) + checkDuplicateClauses(ctx.DBPROPERTIES, "WITH DBPROPERTIES", ctx) + + if (!ctx.PROPERTIES.isEmpty && !ctx.DBPROPERTIES.isEmpty) { + throw new ParseException(s"Either PROPERTIES or DBPROPERTIES is allowed.", ctx) + } + + var properties = ctx.tablePropertyList.asScala.headOption + .map(visitPropertyKeyValues) + .getOrElse(Map.empty) + Option(ctx.comment).map(string).map { + properties += CreateNamespaceStatement.COMMENT_PROPERTY_KEY -> _ + } + ctx.locationSpec.asScala.headOption.map(visitLocationSpec).map { + properties += CreateNamespaceStatement.LOCATION_PROPERTY_KEY -> _ + } + + CreateNamespaceStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.EXISTS != null, + properties) + } + /** * Create a [[ShowNamespacesStatement]] command. */ override def visitShowNamespaces(ctx: ShowNamespacesContext): LogicalPlan = withOrigin(ctx) { + if (ctx.DATABASES != null && ctx.multipartIdentifier != null) { + throw new ParseException(s"FROM/IN operator is not allowed in SHOW DATABASES", ctx) + } + ShowNamespacesStatement( Option(ctx.multipartIdentifier).map(visitMultipartIdentifier), Option(ctx.pattern).map(string)) @@ -2432,6 +2541,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.EXISTS != null) } + /** + * Create a [[UseStatement]] logical plan. + */ + override def visitUse(ctx: UseContext): LogicalPlan = withOrigin(ctx) { + val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier) + UseStatement(ctx.NAMESPACE != null, nameParts) + } + /** * Create a [[ShowTablesStatement]] command. */ @@ -2619,4 +2736,138 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging isExtended) } } + + /** + * Create an [[AnalyzeTableStatement]], or an [[AnalyzeColumnStatement]]. + * Example SQL for analyzing a table or a set of partitions : + * {{{ + * ANALYZE TABLE multi_part_name [PARTITION (partcol1[=val1], partcol2[=val2], ...)] + * COMPUTE STATISTICS [NOSCAN]; + * }}} + * + * Example SQL for analyzing columns : + * {{{ + * ANALYZE TABLE multi_part_name COMPUTE STATISTICS FOR COLUMNS column1, column2; + * }}} + * + * Example SQL for analyzing all columns of a table: + * {{{ + * ANALYZE TABLE multi_part_name COMPUTE STATISTICS FOR ALL COLUMNS; + * }}} + */ + override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) { + def checkPartitionSpec(): Unit = { + if (ctx.partitionSpec != null) { + logWarning("Partition specification is ignored when collecting column statistics: " + + ctx.partitionSpec.getText) + } + } + if (ctx.identifier != null && + ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { + throw new ParseException(s"Expected `NOSCAN` instead of `${ctx.identifier.getText}`", ctx) + } + + val tableName = visitMultipartIdentifier(ctx.multipartIdentifier()) + if (ctx.ALL() != null) { + checkPartitionSpec() + AnalyzeColumnStatement(tableName, None, allColumns = true) + } else if (ctx.identifierSeq() == null) { + val partitionSpec = if (ctx.partitionSpec != null) { + visitPartitionSpec(ctx.partitionSpec) + } else { + Map.empty[String, Option[String]] + } + AnalyzeTableStatement(tableName, partitionSpec, noScan = ctx.identifier != null) + } else { + checkPartitionSpec() + AnalyzeColumnStatement( + tableName, Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) + } + } + + /** + * Create a [[RepairTableStatement]]. + * + * For example: + * {{{ + * MSCK REPAIR TABLE multi_part_name + * }}} + */ + override def visitRepairTable(ctx: RepairTableContext): LogicalPlan = withOrigin(ctx) { + RepairTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) + } + + /** + * Create a [[CacheTableStatement]]. + * + * For example: + * {{{ + * CACHE [LAZY] TABLE multi_part_name + * [OPTIONS tablePropertyList] [[AS] query] + * }}} + */ + override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val query = Option(ctx.query).map(plan) + val tableName = visitMultipartIdentifier(ctx.multipartIdentifier) + if (query.isDefined && tableName.length > 1) { + val catalogAndNamespace = tableName.init + throw new ParseException("It is not allowed to add catalog/namespace " + + s"prefix ${catalogAndNamespace.quoted} to " + + "the table name in CACHE TABLE AS SELECT", ctx) + } + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + CacheTableStatement(tableName, query, ctx.LAZY != null, options) + } + + /** + * Create an [[UncacheTableStatement]] logical plan. + */ + override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { + UncacheTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier), ctx.EXISTS != null) + } + + /** + * Create a [[TruncateTableStatement]] command. + * + * For example: + * {{{ + * TRUNCATE TABLE multi_part_name [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ + override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { + TruncateTableStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + + /** + * A command for users to list the partition names of a table. If partition spec is specified, + * partitions that match the spec are returned. Otherwise an empty result set is returned. + * + * This function creates a [[ShowPartitionsStatement]] logical plan + * + * The syntax of using this command in SQL is: + * {{{ + * SHOW PARTITIONS multi_part_name [partition_spec]; + * }}} + */ + override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { + val table = visitMultipartIdentifier(ctx.multipartIdentifier) + val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + ShowPartitionsStatement(table, partitionKeys) + } + + /** + * Create a [[RefreshTableStatement]]. + * + * For example: + * {{{ + * REFRESH TABLE multi_part_name + * }}} + */ + override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { + RefreshTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) + } } 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 85998e33140d0..b66cae7979416 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 @@ -29,11 +29,20 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.unsafe.types.CalendarInterval /** * Base SQL parsing infrastructure. */ -abstract class AbstractSqlParser extends ParserInterface with Logging { +abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Logging { + + /** + * Creates [[CalendarInterval]] for a given SQL String. Throws [[ParseException]] if the SQL + * string is not a valid interval format. + */ + def parseInterval(sqlText: String): CalendarInterval = parse(sqlText) { parser => + astBuilder.visitSingleInterval(parser.singleInterval()) + } /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => @@ -91,16 +100,16 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) - lexer.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - lexer.ansi = SQLConf.get.ansiEnabled + lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced + lexer.ansi = conf.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) parser.addParseListener(PostProcessor) parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) - parser.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - parser.ansi = SQLConf.get.ansiEnabled + parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced + parser.ansi = conf.ansiEnabled try { try { @@ -134,12 +143,12 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { /** * Concrete SQL parser for Catalyst-only SQL statements. */ -class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser { +class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { val astBuilder = new AstBuilder(conf) } /** For test-only. */ -object CatalystSqlParser extends AbstractSqlParser { +object CatalystSqlParser extends AbstractSqlParser(SQLConf.get) { val astBuilder = new AstBuilder(SQLConf.get) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 89347f4b1f7bf..a3779698a5ac3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -116,7 +116,7 @@ object ParserUtils { var enclosure: Character = null val sb = new StringBuilder(b.length()) - def appendEscapedChar(n: Char) { + def appendEscapedChar(n: Char): Unit = { n match { case '0' => sb.append('\u0000') case '\'' => sb.append('\'') 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 9356d7707104e..d58c607cb87ee 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, 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.connector.catalog.{Identifier, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, SupportsNamespaces, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types._ @@ -382,283 +382,6 @@ case class Join( } } -/** - * Base trait for DataSourceV2 write commands - */ -trait V2WriteCommand extends Command { - def table: NamedRelation - def query: LogicalPlan - - override def children: Seq[LogicalPlan] = Seq(query) - - 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) - } - } - } -} - -/** - * 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 with V2CreateTablePlan { - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * 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 with V2CreateTablePlan { - - override def tableSchema: StructType = query.schema - 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) - } - - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Replace a table with a v2 catalog. - * - * If the table does not exist, and orCreate is true, then it will be created. - * If the table does not exist, and orCreate is false, then an exception will be thrown. - * - * The persisted table will have no contents as a result of this operation. - */ -case class ReplaceTable( - catalog: TableCatalog, - tableName: Identifier, - tableSchema: StructType, - partitioning: Seq[Transform], - properties: Map[String, String], - orCreate: Boolean) extends Command with V2CreateTablePlan { - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Replaces a table from a select query with a v2 catalog. - * - * If the table does not exist, and orCreate is true, then it will be created. - * If the table does not exist, and orCreate is false, then an exception will be thrown. - */ -case class ReplaceTableAsSelect( - catalog: TableCatalog, - tableName: Identifier, - partitioning: Seq[Transform], - query: LogicalPlan, - properties: Map[String, String], - writeOptions: Map[String, String], - orCreate: Boolean) extends Command with V2CreateTablePlan { - - override def tableSchema: StructType = query.schema - 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) - } - - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Append data to an existing table. - */ -case class AppendData( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String], - isByName: Boolean) extends V2WriteCommand - -object AppendData { - def byName( - table: NamedRelation, - df: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): AppendData = { - new AppendData(table, df, writeOptions, isByName = true) - } - - def byPosition( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): AppendData = { - new AppendData(table, query, writeOptions, isByName = false) - } -} - -/** - * Overwrite data matching a filter in an existing table. - */ -case class OverwriteByExpression( - table: NamedRelation, - deleteExpr: Expression, - query: LogicalPlan, - writeOptions: Map[String, String], - isByName: Boolean) extends V2WriteCommand { - override lazy val resolved: Boolean = outputResolved && deleteExpr.resolved -} - -object OverwriteByExpression { - def byName( - table: NamedRelation, - df: LogicalPlan, - deleteExpr: Expression, - writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, df, writeOptions, isByName = true) - } - - def byPosition( - table: NamedRelation, - query: LogicalPlan, - deleteExpr: Expression, - writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, query, writeOptions, isByName = false) - } -} - -/** - * Dynamically overwrite partitions in an existing table. - */ -case class OverwritePartitionsDynamic( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String], - isByName: Boolean) extends V2WriteCommand - -object OverwritePartitionsDynamic { - def byName( - table: NamedRelation, - df: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, df, writeOptions, isByName = true) - } - - def byPosition( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, query, writeOptions, isByName = false) - } -} - -/** - * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. - */ -case class ShowNamespaces( - catalog: SupportsNamespaces, - namespace: Option[Seq[String]], - pattern: Option[String]) extends Command { - override val output: Seq[Attribute] = Seq( - AttributeReference("namespace", StringType, nullable = false)()) -} - -case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { - - override def children: Seq[LogicalPlan] = Seq(table) - - override val output = DescribeTableSchema.describeTableAttributes() -} - -case class DeleteFromTable( - child: LogicalPlan, - condition: Option[Expression]) extends Command { - - override def children: Seq[LogicalPlan] = child :: Nil -} - -/** - * 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 - } - } -} - -/** - * The logical plan of the SHOW TABLE command that works for v2 catalogs. - */ -case class ShowTables( - catalog: TableCatalog, - namespace: Seq[String], - pattern: Option[String]) extends Command { - override val output: Seq[Attribute] = Seq( - AttributeReference("namespace", StringType, nullable = false)(), - AttributeReference("tableName", StringType, nullable = false)()) -} - /** * Insert query result into a directory. * @@ -1233,15 +956,8 @@ case class Deduplicate( override def output: Seq[Attribute] = child.output } -/** A trait used for logical plan nodes that create or replace V2 table definitions. */ -trait V2CreateTablePlan extends LogicalPlan { - def tableName: Identifier - def partitioning: Seq[Transform] - def tableSchema: StructType - - /** - * Creates a copy of this node with the new partitoning transforms. This method is used to - * rewrite the partition transforms normalized according to the table schema. - */ - def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan -} +/** + * A trait to represent the commands that support subqueries. + * This is used to whitelist such commands in the subquery-related checks. + */ +trait SupportsSubquery extends LogicalPlan 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 deleted file mode 100644 index 8c1b54be46cf6..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala +++ /dev/null @@ -1,82 +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.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]) - -trait AlterTableStatement extends ParsedStatement { - val tableName: Seq[String] -} - -/** - * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. - */ -case class AlterTableAddColumnsStatement( - tableName: Seq[String], - columnsToAdd: Seq[QualifiedColType]) extends AlterTableStatement - -/** - * 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 AlterTableStatement - -/** - * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. - */ -case class AlterTableRenameColumnStatement( - tableName: Seq[String], - column: Seq[String], - newName: String) extends AlterTableStatement - -/** - * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL. - */ -case class AlterTableDropColumnsStatement( - tableName: Seq[String], - columnsToDrop: Seq[Seq[String]]) extends AlterTableStatement - -/** - * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterTableSetPropertiesStatement( - tableName: Seq[String], - properties: Map[String, String]) extends AlterTableStatement - -/** - * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterTableUnsetPropertiesStatement( - tableName: Seq[String], - propertyKeys: Seq[String], - ifExists: Boolean) extends AlterTableStatement - -/** - * ALTER TABLE ... SET LOCATION command, as parsed from SQL. - */ -case class AlterTableSetLocationStatement( - tableName: Seq[String], - location: String) extends AlterTableStatement 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 deleted file mode 100644 index 551a0ce44b4d5..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.expressions.Transform -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/DescribeColumnStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala deleted file mode 100644 index 06b9f3e9a9a3e..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala +++ /dev/null @@ -1,23 +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.plans.logical.sql - -case class DescribeColumnStatement( - tableName: Seq[String], - colNameParts: Seq[String], - isExtended: Boolean) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala deleted file mode 100644 index c4210eabe26a3..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala +++ /dev/null @@ -1,50 +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.plans.logical.sql - -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -/** - * An INSERT INTO statement, as parsed from SQL. - * - * @param table the logical plan representing the table. - * @param query the logical plan representing data to write to. - * @param overwrite overwrite existing table or partitions. - * @param partitionSpec a map from the partition key to the partition value (optional). - * If the value is missing, dynamic partition insert will be performed. - * As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS` would have - * Map('a' -> Some('1'), 'b' -> Some('2')), - * and `INSERT INTO tbl PARTITION (a=1, b) AS ...` - * would have Map('a' -> Some('1'), 'b' -> None). - * @param ifPartitionNotExists If true, only write if the partition does not exist. - * Only valid for static partitions. - */ -case class InsertIntoStatement( - table: LogicalPlan, - partitionSpec: Map[String, Option[String]], - query: LogicalPlan, - overwrite: Boolean, - ifPartitionNotExists: Boolean) extends ParsedStatement { - - require(overwrite || !ifPartitionNotExists, - "IF NOT EXISTS is only valid in INSERT OVERWRITE") - require(partitionSpec.values.forall(_.nonEmpty) || !ifPartitionNotExists, - "IF NOT EXISTS is only valid with static partitions") - - override def children: Seq[LogicalPlan] = query :: Nil -} 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 deleted file mode 100644 index 23fc009fecdc2..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala +++ /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.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/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala deleted file mode 100644 index b9767ff9b889a..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.StructType - -/** - * A REPLACE TABLE command, as parsed from SQL. - * - * If the table exists prior to running this command, executing this statement - * will replace the table's metadata and clear the underlying rows from the table. - */ -case class ReplaceTableStatement( - 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], - orCreate: Boolean) extends ParsedStatement - -/** - * A REPLACE TABLE AS SELECT command, as parsed from SQL. - */ -case class ReplaceTableAsSelectStatement( - 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], - orCreate: Boolean) extends ParsedStatement { - - override def children: Seq[LogicalPlan] = Seq(asSelect) -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala deleted file mode 100644 index 95d48a8e80d23..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala +++ /dev/null @@ -1,24 +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.plans.logical.sql - -/** - * A SHOW NAMESPACES statement, as parsed from SQL. - */ -case class ShowNamespacesStatement(namespace: Option[Seq[String]], pattern: Option[String]) - extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala deleted file mode 100644 index d75c4085a974b..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala +++ /dev/null @@ -1,24 +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.plans.logical.sql - -/** - * A SHOW TABLES statement, as parsed from SQL. - */ -case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) - extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala new file mode 100644 index 0000000000000..ef8c922694347 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.catalog.BucketSpec +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.{DataType, StructType} + +/** + * 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]]. + */ +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 +} + +/** + * 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) +} + +/** + * A REPLACE TABLE command, as parsed from SQL. + * + * If the table exists prior to running this command, executing this statement + * will replace the table's metadata and clear the underlying rows from the table. + */ +case class ReplaceTableStatement( + 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], + orCreate: Boolean) extends ParsedStatement + +/** + * A REPLACE TABLE AS SELECT command, as parsed from SQL. + */ +case class ReplaceTableAsSelectStatement( + 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], + orCreate: Boolean) extends ParsedStatement { + + override def children: Seq[LogicalPlan] = Seq(asSelect) +} + + +/** + * 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 + +/** + * 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 + + +/** + * A DROP TABLE statement, as parsed from SQL. + */ +case class DropTableStatement( + tableName: Seq[String], + ifExists: Boolean, + purge: Boolean) extends ParsedStatement + +/** + * A DROP VIEW statement, as parsed from SQL. + */ +case class DropViewStatement( + viewName: Seq[String], + ifExists: Boolean) extends ParsedStatement + +/** + * A DESCRIBE TABLE tbl_name statement, as parsed from SQL. + */ +case class DescribeTableStatement( + tableName: Seq[String], + partitionSpec: TablePartitionSpec, + isExtended: Boolean) extends ParsedStatement + +/** + * A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL. + */ +case class DescribeColumnStatement( + tableName: Seq[String], + colNameParts: Seq[String], + isExtended: Boolean) extends ParsedStatement + +/** + * A DELETE FROM statement, as parsed from SQL. + */ +case class DeleteFromStatement( + tableName: Seq[String], + tableAlias: Option[String], + condition: Option[Expression]) extends ParsedStatement + +/** + * A UPDATE tbl_name statement, as parsed from SQL. + */ +case class UpdateTableStatement( + tableName: Seq[String], + tableAlias: Option[String], + columns: Seq[Seq[String]], + values: Seq[Expression], + condition: Option[Expression]) extends ParsedStatement + +/** + * An INSERT INTO statement, as parsed from SQL. + * + * @param table the logical plan representing the table. + * @param query the logical plan representing data to write to. + * @param overwrite overwrite existing table or partitions. + * @param partitionSpec a map from the partition key to the partition value (optional). + * If the value is missing, dynamic partition insert will be performed. + * As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS` would have + * Map('a' -> Some('1'), 'b' -> Some('2')), + * and `INSERT INTO tbl PARTITION (a=1, b) AS ...` + * would have Map('a' -> Some('1'), 'b' -> None). + * @param ifPartitionNotExists If true, only write if the partition does not exist. + * Only valid for static partitions. + */ +case class InsertIntoStatement( + table: LogicalPlan, + partitionSpec: Map[String, Option[String]], + query: LogicalPlan, + overwrite: Boolean, + ifPartitionNotExists: Boolean) extends ParsedStatement { + + require(overwrite || !ifPartitionNotExists, + "IF NOT EXISTS is only valid in INSERT OVERWRITE") + require(partitionSpec.values.forall(_.nonEmpty) || !ifPartitionNotExists, + "IF NOT EXISTS is only valid with static partitions") + + override def children: Seq[LogicalPlan] = query :: Nil +} + +/** + * A SHOW TABLES statement, as parsed from SQL. + */ +case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) + extends ParsedStatement + +/** + * A CREATE NAMESPACE statement, as parsed from SQL. + */ +case class CreateNamespaceStatement( + namespace: Seq[String], + ifNotExists: Boolean, + properties: Map[String, String]) extends ParsedStatement + +object CreateNamespaceStatement { + val COMMENT_PROPERTY_KEY: String = "comment" + val LOCATION_PROPERTY_KEY: String = "location" +} + +/** + * A SHOW NAMESPACES statement, as parsed from SQL. + */ +case class ShowNamespacesStatement(namespace: Option[Seq[String]], pattern: Option[String]) + extends ParsedStatement + +/** + * A USE statement, as parsed from SQL. + */ +case class UseStatement(isNamespaceSet: Boolean, nameParts: Seq[String]) extends ParsedStatement + +/** + * An ANALYZE TABLE statement, as parsed from SQL. + */ +case class AnalyzeTableStatement( + tableName: Seq[String], + partitionSpec: Map[String, Option[String]], + noScan: Boolean) extends ParsedStatement + +/** + * An ANALYZE TABLE FOR COLUMNS statement, as parsed from SQL. + */ +case class AnalyzeColumnStatement( + tableName: Seq[String], + columnNames: Option[Seq[String]], + allColumns: Boolean) extends ParsedStatement { + require(columnNames.isDefined ^ allColumns, "Parameter `columnNames` or `allColumns` are " + + "mutually exclusive. Only one of them should be specified.") +} + +/** + * A REPAIR TABLE statement, as parsed from SQL + */ +case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement + +/** + * A CACHE TABLE statement, as parsed from SQL + */ +case class CacheTableStatement( + tableName: Seq[String], + plan: Option[LogicalPlan], + isLazy: Boolean, + options: Map[String, String]) extends ParsedStatement + +/** + * An UNCACHE TABLE statement, as parsed from SQL + */ +case class UncacheTableStatement( + tableName: Seq[String], + ifExists: Boolean) extends ParsedStatement + +/** + * A TRUNCATE TABLE statement, as parsed from SQL + */ +case class TruncateTableStatement( + tableName: Seq[String], + partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement + +/** + * A SHOW PARTITIONS statement, as parsed from SQL + */ +case class ShowPartitionsStatement( + tableName: Seq[String], + partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement + +/** + * A REFRESH TABLE statement, as parsed from SQL + */ +case class RefreshTableStatement(tableName: Seq[String]) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala new file mode 100644 index 0000000000000..d80c1c034a867 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.analysis.NamedRelation +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.{DataType, StringType, StructType} + +/** + * Base trait for DataSourceV2 write commands + */ +trait V2WriteCommand extends Command { + def table: NamedRelation + def query: LogicalPlan + + override def children: Seq[LogicalPlan] = Seq(query) + + 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) + } + } + } +} + +/** + * Append data to an existing table. + */ +case class AppendData( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String], + isByName: Boolean) extends V2WriteCommand + +object AppendData { + def byName( + table: NamedRelation, + df: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, df, writeOptions, isByName = true) + } + + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, query, writeOptions, isByName = false) + } +} + +/** + * Overwrite data matching a filter in an existing table. + */ +case class OverwriteByExpression( + table: NamedRelation, + deleteExpr: Expression, + query: LogicalPlan, + writeOptions: Map[String, String], + isByName: Boolean) extends V2WriteCommand { + override lazy val resolved: Boolean = outputResolved && deleteExpr.resolved +} + +object OverwriteByExpression { + def byName( + table: NamedRelation, + df: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, df, writeOptions, isByName = true) + } + + def byPosition( + table: NamedRelation, + query: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, query, writeOptions, isByName = false) + } +} + +/** + * Dynamically overwrite partitions in an existing table. + */ +case class OverwritePartitionsDynamic( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String], + isByName: Boolean) extends V2WriteCommand + +object OverwritePartitionsDynamic { + def byName( + table: NamedRelation, + df: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, df, writeOptions, isByName = true) + } + + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, query, writeOptions, isByName = false) + } +} + + +/** A trait used for logical plan nodes that create or replace V2 table definitions. */ +trait V2CreateTablePlan extends LogicalPlan { + def tableName: Identifier + def partitioning: Seq[Transform] + def tableSchema: StructType + + /** + * Creates a copy of this node with the new partitioning transforms. This method is used to + * rewrite the partition transforms normalized according to the table schema. + */ + def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan +} + +/** + * 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 with V2CreateTablePlan { + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * 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 with V2CreateTablePlan { + + override def tableSchema: StructType = query.schema + 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) + } + + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * Replace a table with a v2 catalog. + * + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + * + * The persisted table will have no contents as a result of this operation. + */ +case class ReplaceTable( + catalog: TableCatalog, + tableName: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + properties: Map[String, String], + orCreate: Boolean) extends Command with V2CreateTablePlan { + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * Replaces a table from a select query with a v2 catalog. + * + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + */ +case class ReplaceTableAsSelect( + catalog: TableCatalog, + tableName: Identifier, + partitioning: Seq[Transform], + query: LogicalPlan, + properties: Map[String, String], + writeOptions: Map[String, String], + orCreate: Boolean) extends Command with V2CreateTablePlan { + + override def tableSchema: StructType = query.schema + 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) + } + + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * The logical plan of the CREATE NAMESPACE command that works for v2 catalogs. + */ +case class CreateNamespace( + catalog: SupportsNamespaces, + namespace: Seq[String], + ifNotExists: Boolean, + properties: Map[String, String]) extends Command + +/** + * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. + */ +case class ShowNamespaces( + catalog: SupportsNamespaces, + namespace: Option[Seq[String]], + pattern: Option[String]) extends Command { + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)()) +} + +/** + * The logical plan of the DESCRIBE TABLE command that works for v2 tables. + */ +case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() +} + +/** + * The logical plan of the DELETE FROM command that works for v2 tables. + */ +case class DeleteFromTable( + table: LogicalPlan, + condition: Option[Expression]) extends Command with SupportsSubquery { + override def children: Seq[LogicalPlan] = table :: Nil +} + +/** + * The logical plan of the UPDATE TABLE command that works for v2 tables. + */ +case class UpdateTable( + table: LogicalPlan, + columns: Seq[Expression], + values: Seq[Expression], + condition: Option[Expression]) extends Command with SupportsSubquery { + override def children: Seq[LogicalPlan] = table :: Nil +} + +/** + * The logical plan of the DROP TABLE command that works for v2 tables. + */ +case class DropTable( + catalog: TableCatalog, + ident: Identifier, + ifExists: Boolean) extends Command + +/** + * The logical plan of the ALTER TABLE command that works for v2 tables. + */ +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 + } + } +} + +/** + * The logical plan of the SHOW TABLE command that works for v2 catalogs. + */ +case class ShowTables( + catalog: TableCatalog, + namespace: Seq[String], + pattern: Option[String]) extends Command { + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)(), + AttributeReference("tableName", StringType, nullable = false)()) +} + +/** + * The logical plan of the USE/USE NAMESPACE command that works for v2 catalogs. + */ +case class SetCatalogAndNamespace( + catalogManager: CatalogManager, + catalogName: Option[String], + namespace: Option[Seq[String]]) extends Command + +/** + * The logical plan of the REFRESH TABLE command that works for v2 catalogs. + */ +case class RefreshTable( + catalog: TableCatalog, + ident: Identifier) extends Command 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 00fb81e361fdf..ba1eeb38e247e 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 @@ -27,9 +27,8 @@ import org.json4s.JsonAST._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.IdentifierWithDatabase import org.apache.spark.sql.catalyst.ScalaReflection._ -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, FunctionResource} import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ @@ -780,9 +779,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private def shouldConvertToJson(product: Product): Boolean = product match { case exprId: ExprId => true case field: StructField => true - case id: TableIdentifier => true + case id: IdentifierWithDatabase => true case join: JoinType => true - case id: FunctionIdentifier => true case spec: BucketSpec => true case catalog: CatalogTable => true case partition: Partitioning => true 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 3cf310a0a9683..8dd541b1e4381 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit._ import scala.util.control.NonFatal import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** * Helper functions for converting between internal and external date and time representations. @@ -286,7 +286,7 @@ object DateTimeUtils { i += 1 } } else { - if (b == ':' || b == ' ') { + if (i < segments.length && (b == ':' || b == ' ')) { segments(i) = currentSegmentValue currentSegmentValue = 0 i += 1 @@ -461,6 +461,14 @@ object DateTimeUtils { (MICROSECONDS.toSeconds(localTimestamp(microsec, timeZone)) % 60).toInt } + /** + * Returns the seconds part and its fractional part with microseconds. + */ + def getSecondsWithFraction(microsec: SQLTimestamp, timeZone: TimeZone): Decimal = { + val secFrac = localTimestamp(microsec, timeZone) % (MILLIS_PER_MINUTE * MICROS_PER_MILLIS) + Decimal(secFrac, 8, 6) + } + /** * Returns seconds, including fractional parts, multiplied by 1000. The timestamp * is expressed in microseconds since the epoch. @@ -942,4 +950,20 @@ object DateTimeUtils { None } } + + /** + * Subtracts two dates. + * @param endDate - the end date, exclusive + * @param startDate - the start date, inclusive + * @return an interval between two dates. The interval can be negative + * if the end date is before the start date. + */ + def subtractDates(endDate: SQLDate, startDate: SQLDate): CalendarInterval = { + val period = Period.between( + LocalDate.ofEpochDay(startDate), + LocalDate.ofEpochDay(endDate)) + val months = period.getMonths + 12 * period.getYears + val microseconds = period.getDays * MICROS_PER_DAY + new CalendarInterval(months, microseconds) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala new file mode 100644 index 0000000000000..14fd153e15f58 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.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.catalyst.util + +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.CalendarInterval + +object IntervalUtils { + final val MONTHS_PER_YEAR: Int = 12 + final val MONTHS_PER_QUARTER: Byte = 3 + final val YEARS_PER_MILLENNIUM: Int = 1000 + final val YEARS_PER_CENTURY: Int = 100 + final val YEARS_PER_DECADE: Int = 10 + final val MICROS_PER_HOUR: Long = + DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + final val MICROS_PER_MINUTE: Long = + DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + final val DAYS_PER_MONTH: Byte = 30 + final val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + /* 365.25 days per year assumes leap year every four years */ + final val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 + + def getYears(interval: CalendarInterval): Int = { + interval.months / MONTHS_PER_YEAR + } + + def getMillenniums(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_MILLENNIUM + } + + def getCenturies(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_CENTURY + } + + def getDecades(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_DECADE + } + + def getMonths(interval: CalendarInterval): Byte = { + (interval.months % MONTHS_PER_YEAR).toByte + } + + def getQuarters(interval: CalendarInterval): Byte = { + (getMonths(interval) / MONTHS_PER_QUARTER + 1).toByte + } + + def getDays(interval: CalendarInterval): Long = { + interval.microseconds / DateTimeUtils.MICROS_PER_DAY + } + + def getHours(interval: CalendarInterval): Byte = { + ((interval.microseconds % DateTimeUtils.MICROS_PER_DAY) / MICROS_PER_HOUR).toByte + } + + def getMinutes(interval: CalendarInterval): Byte = { + ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte + } + + def getMicroseconds(interval: CalendarInterval): Long = { + interval.microseconds % MICROS_PER_MINUTE + } + + def getSeconds(interval: CalendarInterval): Decimal = { + Decimal(getMicroseconds(interval), 8, 6) + } + + def getMilliseconds(interval: CalendarInterval): Decimal = { + Decimal(getMicroseconds(interval), 8, 3) + } + + // Returns total number of seconds with microseconds fractional part in the given interval. + def getEpoch(interval: CalendarInterval): Decimal = { + var result = interval.microseconds + result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) + result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) + Decimal(result, 18, 6) + } + + /** + * Converts a string to [[CalendarInterval]] case-insensitively. + * + * @throws IllegalArgumentException if the input string is not in valid interval format. + */ + def fromString(str: String): CalendarInterval = { + if (str == null) throw new IllegalArgumentException("Interval string cannot be null") + try { + CatalystSqlParser.parseInterval(str) + } catch { + case e: ParseException => + val ex = new IllegalArgumentException(s"Invalid interval string: $str\n" + e.message) + ex.setStackTrace(e.getStackTrace) + throw ex + } + } + + /** + * A safe version of `fromString`. It returns null for invalid input string. + */ + def safeFromString(str: String): CalendarInterval = { + try { + fromString(str) + } catch { + case _: IllegalArgumentException => null + } + } +} 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 2a03f85ab594b..3a0490d07733d 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 @@ -159,14 +159,72 @@ class QuantileSummaries( other.shallowCopy } else { // Merge the two buffers. - // The GK algorithm is a bit unclear about it, but it seems there is no need to adjust the - // statistics during the merging: the invariants are still respected after the merge. - // TODO: could replace full sort by ordered merge, the two lists are known to be sorted - // already. - val res = (sampled ++ other.sampled).sortBy(_.value) - val comp = compressImmut(res, mergeThreshold = 2 * relativeError * count) - new QuantileSummaries( - other.compressThreshold, other.relativeError, comp, other.count + count, true) + // The GK algorithm is a bit unclear about it, but we need to adjust the statistics during the + // merging. The main idea is that samples that come from one side will suffer from the lack of + // precision of the other. + // As a concrete example, take two QuantileSummaries whose samples (value, g, delta) are: + // `a = [(0, 1, 0), (20, 99, 0)]` and `b = [(10, 1, 0), (30, 49, 0)]` + // This means `a` has 100 values, whose minimum is 0 and maximum is 20, + // while `b` has 50 values, between 10 and 30. + // The resulting samples of the merge will be: + // a+b = [(0, 1, 0), (10, 1, ??), (20, 99, ??), (30, 49, 0)] + // The values of `g` do not change, as they represent the minimum number of values between two + // consecutive samples. The values of `delta` should be adjusted, however. + // Take the case of the sample `10` from `b`. In the original stream, it could have appeared + // right after `0` (as expressed by `g=1`) or right before `20`, so `delta=99+0-1=98`. + // In the GK algorithm's style of working in terms of maximum bounds, one can observe that the + // maximum additional uncertainty over samples comming from `b` is `max(g_a + delta_a) = + // floor(2 * eps_a * n_a)`. Likewise, additional uncertainty over samples from `a` is + // `floor(2 * eps_b * n_b)`. + // Only samples that interleave the other side are affected. That means that samples from + // one side that are lesser (or greater) than all samples from the other side are just copied + // unmodifed. + // If the merging instances have different `relativeError`, the resulting instance will cary + // the largest one: `eps_ab = max(eps_a, eps_b)`. + // The main invariant of the GK algorithm is kept: + // `max(g_ab + delta_ab) <= floor(2 * eps_ab * (n_a + n_b))` since + // `max(g_ab + delta_ab) <= floor(2 * eps_a * n_a) + floor(2 * eps_b * n_b)` + // Finally, one can see how the `insert(x)` operation can be expressed as `merge([(x, 1, 0])` + + val mergedSampled = new ArrayBuffer[Stats]() + val mergedRelativeError = math.max(relativeError, other.relativeError) + val mergedCount = count + other.count + val additionalSelfDelta = math.floor(2 * other.relativeError * other.count).toLong + val additionalOtherDelta = math.floor(2 * relativeError * count).toLong + + // Do a merge of two sorted lists until one of the lists is fully consumed + var selfIdx = 0 + var otherIdx = 0 + while (selfIdx < sampled.length && otherIdx < other.sampled.length) { + val selfSample = sampled(selfIdx) + val otherSample = other.sampled(otherIdx) + + // Detect next sample + val (nextSample, additionalDelta) = if (selfSample.value < otherSample.value) { + selfIdx += 1 + (selfSample, if (otherIdx > 0) additionalSelfDelta else 0) + } else { + otherIdx += 1 + (otherSample, if (selfIdx > 0) additionalOtherDelta else 0) + } + + // Insert it + mergedSampled += nextSample.copy(delta = nextSample.delta + additionalDelta) + } + + // Copy the remaining samples from the other list + // (by construction, at most one `while` loop will run) + while (selfIdx < sampled.length) { + mergedSampled += sampled(selfIdx) + selfIdx += 1 + } + while (otherIdx < other.sampled.length) { + mergedSampled += other.sampled(otherIdx) + otherIdx += 1 + } + + val comp = compressImmut(mergedSampled, 2 * mergedRelativeError * mergedCount) + new QuantileSummaries(other.compressThreshold, mergedRelativeError, comp, mergedCount, true) } } 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 a14ae540f5056..3bda9a2a1fc48 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 @@ -65,16 +65,16 @@ object StringUtils extends Logging { "(?s)" + out.result() // (?s) enables dotall mode, causing "." to match new lines } - // "true", "yes", "1", "false", "no", "0", and unique prefixes of these strings are accepted. private[this] val trueStrings = - Set("true", "tru", "tr", "t", "yes", "ye", "y", "on", "1").map(UTF8String.fromString) + Set("t", "true", "y", "yes", "1").map(UTF8String.fromString) private[this] val falseStrings = - Set("false", "fals", "fal", "fa", "f", "no", "n", "off", "of", "0").map(UTF8String.fromString) + Set("f", "false", "n", "no", "0").map(UTF8String.fromString) // scalastyle:off caselocale - def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.toLowerCase.trim()) - def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.toLowerCase.trim()) + def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.toLowerCase) + + def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.toLowerCase) // scalastyle:on caselocale /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala similarity index 56% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala index 7631efedf46af..1ae15df29d6e7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala @@ -15,18 +15,19 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.test +package org.apache.spark.sql.catalyst.util.postgreSQL -import java.io.File +import org.apache.spark.unsafe.types.UTF8String -import org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax -import org.apache.hive.hcatalog.data.JsonSerDe +object StringUtils { + // "true", "yes", "1", "false", "no", "0", and unique prefixes of these strings are accepted. + private[this] val trueStrings = + Set("true", "tru", "tr", "t", "yes", "ye", "y", "on", "1").map(UTF8String.fromString) -object HiveTestUtils { + private[this] val falseStrings = + Set("false", "fals", "fal", "fa", "f", "no", "n", "off", "of", "0").map(UTF8String.fromString) - val getHiveContribJar: File = - new File(classOf[UDAFExampleMax].getProtectionDomain.getCodeSource.getLocation.getPath) + def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s) - val getHiveHcatalogCoreJar: File = - new File(classOf[JsonSerDe].getProtectionDomain.getCodeSource.getLocation.getPath) + def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 3cf8cec84d0c3..c9d050768c15e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -21,26 +21,39 @@ import scala.collection.mutable import scala.util.control.NonFatal import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.internal.SQLConf /** * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow * the caller to look up a catalog by name. + * + * There are still many commands (e.g. ANALYZE TABLE) that do not support v2 catalog API. They + * ignore the current catalog and blindly go to the v1 `SessionCatalog`. To avoid tracking current + * namespace in both `SessionCatalog` and `CatalogManger`, we let `CatalogManager` to set/get + * current database of `SessionCatalog` when the current catalog is the session catalog. */ +// TODO: all commands should look up table from the current catalog. The `SessionCatalog` doesn't +// need to track current database at all. private[sql] -class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends Logging { +class CatalogManager( + conf: SQLConf, + defaultSessionCatalog: CatalogPlugin, + val v1SessionCatalog: SessionCatalog) extends Logging { + import CatalogManager.SESSION_CATALOG_NAME private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] def catalog(name: String): CatalogPlugin = synchronized { - if (name.equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME)) { + if (name.equalsIgnoreCase(SESSION_CATALOG_NAME)) { v2SessionCatalog } else { catalogs.getOrElseUpdate(name, Catalogs.load(name, conf)) } } - def defaultCatalog: Option[CatalogPlugin] = { + private def defaultCatalog: Option[CatalogPlugin] = { conf.defaultV2Catalog.flatMap { catalogName => try { Some(catalog(catalogName)) @@ -53,7 +66,7 @@ class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends } private def loadV2SessionCatalog(): CatalogPlugin = { - Catalogs.load(CatalogManager.SESSION_CATALOG_NAME, conf) match { + Catalogs.load(SESSION_CATALOG_NAME, conf) match { case extension: CatalogExtension => extension.setDelegateCatalog(defaultSessionCatalog) extension @@ -61,12 +74,19 @@ class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends } } - // If the V2_SESSION_CATALOG config is specified, we try to instantiate the user-specified v2 - // session catalog. Otherwise, return the default session catalog. - def v2SessionCatalog: CatalogPlugin = { - conf.getConf(SQLConf.V2_SESSION_CATALOG).map { customV2SessionCatalog => + /** + * If the V2_SESSION_CATALOG config is specified, we try to instantiate the user-specified v2 + * session catalog. Otherwise, return the default session catalog. + * + * 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 + */ + private def v2SessionCatalog: CatalogPlugin = { + conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { - catalogs.getOrElseUpdate(CatalogManager.SESSION_CATALOG_NAME, loadV2SessionCatalog()) + catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) } catch { case NonFatal(_) => logError( @@ -85,36 +105,53 @@ class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends def currentNamespace: Array[String] = synchronized { _currentNamespace.getOrElse { - currentCatalog.map { catalogName => - getDefaultNamespace(catalog(catalogName)) - }.getOrElse(Array("default")) // The builtin catalog use "default" as the default database. + if (currentCatalog.name() == SESSION_CATALOG_NAME) { + Array(v1SessionCatalog.getCurrentDatabase) + } else { + getDefaultNamespace(currentCatalog) + } } } def setCurrentNamespace(namespace: Array[String]): Unit = synchronized { - _currentNamespace = Some(namespace) + if (currentCatalog.name() == SESSION_CATALOG_NAME) { + if (namespace.length != 1) { + throw new NoSuchNamespaceException(namespace) + } + v1SessionCatalog.setCurrentDatabase(namespace.head) + } else { + _currentNamespace = Some(namespace) + } } - private var _currentCatalog: Option[String] = None + private var _currentCatalogName: Option[String] = None - // Returns the name of current catalog. None means the current catalog is the builtin catalog. - def currentCatalog: Option[String] = synchronized { - _currentCatalog.orElse(conf.defaultV2Catalog) + def currentCatalog: CatalogPlugin = synchronized { + _currentCatalogName.map(catalogName => catalog(catalogName)) + .orElse(defaultCatalog) + .getOrElse(v2SessionCatalog) } def setCurrentCatalog(catalogName: String): Unit = synchronized { - _currentCatalog = Some(catalogName) - _currentNamespace = None + // `setCurrentCatalog` is noop if it doesn't switch to a different catalog. + if (currentCatalog.name() != catalogName) { + _currentCatalogName = Some(catalogName) + _currentNamespace = None + // Reset the current database of v1 `SessionCatalog` when switching current catalog, so that + // when we switch back to session catalog, the current namespace definitely is ["default"]. + v1SessionCatalog.setCurrentDatabase(SessionCatalog.DEFAULT_DATABASE) + } } // Clear all the registered catalogs. Only used in tests. private[sql] def reset(): Unit = synchronized { catalogs.clear() _currentNamespace = None - _currentCatalog = None + _currentCatalogName = None + v1SessionCatalog.setCurrentDatabase(SessionCatalog.DEFAULT_DATABASE) } } private[sql] object CatalogManager { - val SESSION_CATALOG_NAME: String = "session" + val SESSION_CATALOG_NAME: String = "spark_catalog" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 031e0586a5f0f..4e5341839a725 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.types.StructType @@ -93,9 +94,21 @@ private[sql] object CatalogV2Implicits { } implicit class MultipartIdentifierHelper(parts: Seq[String]) { - def quoted: String = parts.map(quote).mkString(".") + if (parts.isEmpty) { + throw new AnalysisException("multi-part identifier cannot be empty.") + } def asIdentifier: Identifier = Identifier.of(parts.init.toArray, parts.last) + + def asTableIdentifier: TableIdentifier = parts match { + case Seq(tblName) => TableIdentifier(tblName) + case Seq(dbName, tblName) => TableIdentifier(tblName, Some(dbName)) + case _ => + throw new AnalysisException( + s"$quoted is not a valid TableIdentifier as it has more than 2 name parts.") + } + + def quoted: String = parts.map(quote).mkString(".") } private def quote(part: String): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index a61a55b222858..6d8c6f8456412 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -21,8 +21,11 @@ import java.util import java.util.Collections import scala.collection.JavaConverters._ +import scala.collection.mutable -import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} @@ -220,4 +223,59 @@ private[sql] object CatalogV2Util { case _: NoSuchDatabaseException => None case _: NoSuchNamespaceException => None } + + def isSessionCatalog(catalog: CatalogPlugin): Boolean = { + catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME) + } + + 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 + } + + def createAlterTable( + originalNameParts: Seq[String], + catalog: CatalogPlugin, + tableName: Seq[String], + changes: Seq[TableChange]): AlterTable = { + val tableCatalog = catalog.asTableCatalog + val ident = tableName.asIdentifier + val unresolved = UnresolvedV2Relation(originalNameParts, tableCatalog, ident) + AlterTable(tableCatalog, ident, unresolved, changes) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index baea8204d9be4..26ba93e57fc64 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.connector.catalog -import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.TableIdentifier @@ -29,22 +28,9 @@ private[sql] trait LookupCatalog extends Logging { protected val catalogManager: CatalogManager /** - * 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] = catalogManager.defaultCatalog - - /** - * 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 + * Returns the current catalog set. */ - def sessionCatalog: CatalogPlugin = catalogManager.v2SessionCatalog + def currentCatalog: CatalogPlugin = catalogManager.currentCatalog /** * Extract catalog plugin and remaining identifier names. @@ -65,35 +51,32 @@ private[sql] trait LookupCatalog extends Logging { } } - type CatalogObjectIdentifier = (Option[CatalogPlugin], Identifier) - /** - * Extract catalog and identifier from a multi-part identifier with the default catalog if needed. + * Extract catalog and identifier from a multi-part identifier with the current catalog if needed. */ object CatalogObjectIdentifier { - def unapply(parts: Seq[String]): Some[CatalogObjectIdentifier] = parts match { + def unapply(parts: Seq[String]): Some[(CatalogPlugin, Identifier)] = parts match { case CatalogAndIdentifier(maybeCatalog, nameParts) => Some(( - maybeCatalog.orElse(defaultCatalog), + maybeCatalog.getOrElse(currentCatalog), Identifier.of(nameParts.init.toArray, nameParts.last) )) } } - type CatalogNamespace = (Option[CatalogPlugin], Seq[String]) - /** - * Extract catalog and namespace from a multi-part identifier with the default catalog if needed. + * Extract catalog and namespace from a multi-part identifier with the current catalog if needed. * Catalog name takes precedence over namespaces. */ - object CatalogNamespace { - def unapply(parts: Seq[String]): Some[CatalogNamespace] = parts match { + object CatalogAndNamespace { + def unapply(parts: Seq[String]): Some[(CatalogPlugin, Option[Seq[String]])] = parts match { case Seq(catalogName, tail @ _*) => try { - Some((Some(catalogManager.catalog(catalogName)), tail)) + Some( + (catalogManager.catalog(catalogName), if (tail.isEmpty) { None } else { Some(tail) })) } catch { case _: CatalogNotFoundException => - Some((defaultCatalog, parts)) + Some((currentCatalog, Some(parts))) } } } @@ -105,7 +88,7 @@ private[sql] trait LookupCatalog extends Logging { */ object AsTableIdentifier { def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { - case CatalogAndIdentifier(None, names) if defaultCatalog.isEmpty => + case CatalogAndIdentifier(None, names) if CatalogV2Util.isSessionCatalog(currentCatalog) => names match { case Seq(name) => Some(TableIdentifier(name)) @@ -132,4 +115,19 @@ private[sql] trait LookupCatalog extends Logging { None } } + + /** + * Extract catalog and the rest name parts from a multi-part identifier. + */ + object CatalogAndIdentifierParts { + def unapply(nameParts: Seq[String]): Some[(CatalogPlugin, Seq[String])] = { + assert(nameParts.nonEmpty) + try { + Some((catalogManager.catalog(nameParts.head), nameParts.tail)) + } catch { + case _: CatalogNotFoundException => + Some((currentCatalog, nameParts)) + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala 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 0ec661fc16c88..a228d9f064a1e 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 @@ -36,6 +36,7 @@ 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.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -89,11 +90,16 @@ object SQLConf { } def withExistingConf[T](conf: SQLConf)(f: => T): T = { + val old = existingConf.get() existingConf.set(conf) try { f } finally { - existingConf.remove() + if (old != null) { + existingConf.set(old) + } else { + existingConf.remove() + } } } @@ -343,35 +349,35 @@ object SQLConf { .checkValue(_ > 0, "The value of spark.sql.shuffle.partitions must be positive") .createWithDefault(200) + val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") + .doc("When true, enable adaptive query execution.") + .booleanConf + .createWithDefault(false) + val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE = buildConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize") .doc("The target post-shuffle input size in bytes of a task.") .bytesConf(ByteUnit.BYTE) .createWithDefault(64 * 1024 * 1024) - val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") - .doc("When true, enable adaptive query execution.") - .booleanConf - .createWithDefault(false) - - val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = - buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") - .doc("The relation with a non-empty partition ratio lower than this config will not be " + - "considered as the build side of a broadcast-hash join in adaptive execution regardless " + - "of its size.") - .doubleConf - .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.") - .createWithDefault(0.2) + val FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED = + buildConf("spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled") + .doc("Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks " + + "one by one, fetching continuous shuffle blocks for the same map task in batch can " + + "reduce IO and improve performance. Note, this feature also depends on a relocatable " + + "serializer and the concatenation support codec in use.") + .booleanConf + .createWithDefault(true) val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = - buildConf("spark.sql.adaptive.reducePostShufflePartitions.enabled") + buildConf("spark.sql.adaptive.shuffle.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") + buildConf("spark.sql.adaptive.shuffle.minNumPostShufflePartitions") .doc("The advisory minimum number of post-shuffle partitions used in adaptive execution.") .intConf .checkValue(_ > 0, "The minimum shuffle partition number " + @@ -379,7 +385,7 @@ object SQLConf { .createWithDefault(1) val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = - buildConf("spark.sql.adaptive.maxNumPostShufflePartitions") + buildConf("spark.sql.adaptive.shuffle.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") @@ -388,6 +394,23 @@ object SQLConf { "must be a positive integer.") .createOptional + val OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED = + buildConf("spark.sql.adaptive.shuffle.optimizedLocalShuffleReader.enabled") + .doc("When true and adaptive execution is enabled, this enables the optimization of" + + " converting the shuffle reader to local shuffle reader for the shuffle exchange" + + " of the broadcast hash join in probe side.") + .booleanConf + .createWithDefault(true) + + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = + buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") + .doc("The relation with a non-empty partition ratio lower than this config will not be " + + "considered as the build side of a broadcast-hash join in adaptive execution regardless " + + "of its size.") + .doubleConf + .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.") + .createWithDefault(0.2) + val SUBEXPRESSION_ELIMINATION_ENABLED = buildConf("spark.sql.subexpressionElimination.enabled") .internal() @@ -603,8 +626,8 @@ object SQLConf { .createWithDefault("snappy") val ORC_IMPLEMENTATION = buildConf("spark.sql.orc.impl") - .doc("When native, use the native version of ORC support instead of the ORC library in Hive " + - "1.2.1. It is 'hive' by default prior to Spark 2.4.") + .doc("When native, use the native version of ORC support instead of the ORC library in Hive." + + "It is 'hive' by default prior to Spark 2.4.") .internal() .stringConf .checkValues(Set("hive", "native")) @@ -832,6 +855,17 @@ object SQLConf { .intConf .createWithDefault(10000) + val IGNORE_DATA_LOCALITY = + buildConf("spark.sql.sources.ignoreDataLocality.enabled") + .doc("If true, Spark will not fetch the block locations for each file on " + + "listing files. This speeds up file listing, but the scheduler cannot " + + "schedule tasks to take advantage of data locality. It can be particularly " + + "useful if data is read from a remote cluster so the scheduler could never " + + "take advantage of locality anyway.") + .internal() + .booleanConf + .createWithDefault(false) + // Whether to automatically resolve ambiguity in join conditions for self-joins. // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = @@ -946,7 +980,9 @@ object SQLConf { .createWithDefault(true) 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.") + .doc("The maximum number of bytes to pack into a single partition when reading files. " + + "This configuration is effective only when using file-based sources such as Parquet, JSON " + + "and ORC.") .bytesConf(ByteUnit.BYTE) .createWithDefault(128 * 1024 * 1024) // parquet.block.size @@ -955,19 +991,24 @@ object SQLConf { .doc("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's better to" + " over estimated, then the partitions with small files will be faster than partitions with" + - " bigger files (which is scheduled first).") + " bigger files (which is scheduled first). This configuration is effective only when using" + + " file-based sources such as Parquet, JSON and ORC.") .longConf .createWithDefault(4 * 1024 * 1024) val IGNORE_CORRUPT_FILES = buildConf("spark.sql.files.ignoreCorruptFiles") .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + - "encountering corrupted files and the contents that have been read will still be returned.") + "encountering corrupted files and the contents that have been read will still be returned. " + + "This configuration is effective only when using file-based sources such as Parquet, JSON " + + "and ORC.") .booleanConf .createWithDefault(false) val IGNORE_MISSING_FILES = buildConf("spark.sql.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. " + + "This configuration is effective only when using file-based sources such as Parquet, JSON " + + "and ORC.") .booleanConf .createWithDefault(false) @@ -1153,6 +1194,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val JSON_GENERATOR_IGNORE_NULL_FIELDS = + buildConf("spark.sql.jsonGenerator.ignoreNullFields") + .doc("Whether to ignore null fields when generating JSON objects in JSON data source and " + + "JSON functions such as to_json. " + + "If false, it generates null for null fields in JSON objects.") + .booleanConf + .createWithDefault(true) + val FILE_SINK_LOG_DELETION = buildConf("spark.sql.streaming.fileSink.log.deletion") .internal() .doc("Whether to delete the expired log files in file stream sink.") @@ -1589,12 +1638,22 @@ object SQLConf { .booleanConf .createWithDefault(false) - val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") - .internal() - .doc("When true, will perform integral division with the / operator " + - "if both sides are integral types. This is for PostgreSQL test cases only.") - .booleanConf - .createWithDefault(false) + object Dialect extends Enumeration { + val SPARK, POSTGRESQL = Value + } + + val DIALECT = + buildConf("spark.sql.dialect") + .doc("The specific features of the SQL language to be adopted, which are available when " + + "accessing the given database. Currently, Spark supports two database dialects, `Spark` " + + "and `PostgreSQL`. With `PostgreSQL` dialect, Spark will: " + + "1. perform integral division with the / operator if both sides are integral types; " + + "2. accept \"true\", \"yes\", \"1\", \"false\", \"no\", \"0\", and unique prefixes as " + + "input and trim input for the boolean data type.") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValues(Dialect.values.map(_.toString)) + .createWithDefault(Dialect.SPARK.toString) val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") @@ -1643,7 +1702,7 @@ object SQLConf { "implementation class names for which Data Source V2 code path is disabled. These data " + "sources will fallback to Data Source V1 code path.") .stringConf - .createWithDefault("") + .createWithDefault("kafka") 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" + @@ -1705,7 +1764,7 @@ object SQLConf { .stringConf .transform(_.toUpperCase(Locale.ROOT)) .checkValues(StoreAssignmentPolicy.values.map(_.toString)) - .createOptional + .createWithDefault(StoreAssignmentPolicy.ANSI.toString) val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + @@ -1844,7 +1903,7 @@ object SQLConf { .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + "The size function returns null for null input if the flag is disabled.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED = buildConf("spark.sql.legacy.replaceDatabricksSparkAvro.enabled") @@ -1955,11 +2014,14 @@ object SQLConf { .stringConf .createOptional - val V2_SESSION_CATALOG = buildConf("spark.sql.catalog.session") - .doc("A catalog implementation that will be used in place of the Spark built-in session " + - "catalog for v2 operations. The implementation may extend `CatalogExtension` to be " + - "passed the Spark built-in session catalog, so that it may delegate calls to the " + - "built-in session catalog.") + val V2_SESSION_CATALOG_IMPLEMENTATION = + buildConf(s"spark.sql.catalog.$SESSION_CATALOG_NAME") + .doc("A catalog implementation that will be used as the v2 interface to Spark's built-in " + + s"v1 catalog: $SESSION_CATALOG_NAME. This catalog shares its identifier namespace with " + + s"the $SESSION_CATALOG_NAME and must be consistent with it; for example, if a table can " + + s"be loaded by the $SESSION_CATALOG_NAME, this catalog must also return the table " + + s"metadata. To delegate operations to the $SESSION_CATALOG_NAME, implementations can " + + "extend 'CatalogExtension'.") .stringConf .createOptional @@ -1979,6 +2041,15 @@ object SQLConf { .doc("When true, the ArrayExists will follow the three-valued boolean logic.") .booleanConf .createWithDefault(true) + + val ADDITIONAL_REMOTE_REPOSITORIES = + buildConf("spark.sql.additionalRemoteRepositories") + .doc("A comma-delimited string config of the optional additional remote Maven mirror " + + "repositories. This is only used for downloading Hive jars in IsolatedClientLoader " + + "if the default Maven Central repo is unreachable.") + .stringConf + .createWithDefault( + "https://maven-central.storage-download.googleapis.com/repos/central/data/") } /** @@ -2091,18 +2162,18 @@ class SQLConf extends Serializable with Logging { def numShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS) - def targetPostShuffleInputSize: Long = - getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) - def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) + def targetPostShuffleInputSize: Long = getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) + + def fetchShuffleBlocksInBatchEnabled: Boolean = getConf(FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED) + def nonEmptyPartitionRatioForBroadcastJoin: Double = getConf(NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN) def reducePostShufflePartitionsEnabled: Boolean = getConf(REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED) - def minNumPostShufflePartitions: Int = - getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) + def minNumPostShufflePartitions: Int = getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) def maxNumPostShufflePartitions: Int = getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS).getOrElse(numShufflePartitions) @@ -2323,6 +2394,8 @@ class SQLConf extends Serializable with Logging { def sessionLocalTimeZone: String = getConf(SQLConf.SESSION_LOCAL_TIMEZONE) + def jsonGeneratorIgnoreNullFields: Boolean = getConf(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS) + def parallelFileListingInStatsComputation: Boolean = getConf(SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION) @@ -2418,8 +2491,6 @@ 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) @@ -2428,11 +2499,13 @@ class SQLConf extends Serializable with Logging { def partitionOverwriteMode: PartitionOverwriteMode.Value = PartitionOverwriteMode.withName(getConf(PARTITION_OVERWRITE_MODE)) - def storeAssignmentPolicy: Option[StoreAssignmentPolicy.Value] = - getConf(STORE_ASSIGNMENT_POLICY).map(StoreAssignmentPolicy.withName) + def storeAssignmentPolicy: StoreAssignmentPolicy.Value = + StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) def ansiEnabled: Boolean = getConf(ANSI_ENABLED) + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString() + def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) def serializerNestedSchemaPruningEnabled: Boolean = @@ -2475,6 +2548,8 @@ class SQLConf extends Serializable with Logging { def defaultV2Catalog: Option[String] = getConf(DEFAULT_V2_CATALOG) + def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 3a10a56f6937f..ad1d6b62ef3a1 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 @@ -456,6 +456,8 @@ object DataType { true } + case (_: NullType, _) if storeAssignmentPolicy == ANSI => true + case (w: AtomicType, r: AtomicType) if storeAssignmentPolicy == ANSI => if (!Cast.canANSIStoreAssign(w, r)) { addError(s"Cannot safely cast '$context': $w to $r") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala new file mode 100644 index 0000000000000..ac18b0f79b5f3 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} + +import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObject, JString, JValue} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.encoders.{ExamplePoint, ExamplePointUDT} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * Test suite for [[Row]] JSON serialization. + */ +class RowJsonSuite extends SparkFunSuite { + private val schema = new StructType() + .add("c1", "string") + .add("c2", IntegerType) + + private def testJson(name: String, value: Any, dt: DataType, expected: JValue): Unit = { + test(name) { + val row = new GenericRowWithSchema(Array(value), new StructType().add("a", dt)) + assert(row.jsonValue === JObject("a" -> expected)) + } + } + + private def testJson(value: Any, dt: DataType, expected: JValue): Unit = { + testJson(s"$dt $value", value, dt, expected) + } + + // Nulls + private def testJsonNull(dt: DataType, expected: JValue): Unit = { + testJson(null, dt, JNull) + } + testJsonNull(IntegerType, JNull) + testJsonNull(FloatType, JNull) + testJsonNull(ArrayType(DoubleType, containsNull = true), JNull) + + // Primitives + testJson(true, BooleanType, JBool(true)) + testJson(false, BooleanType, JBool(false)) + testJson(23.toByte, ByteType, JLong(23)) + testJson(-126.toByte, ByteType, JLong(-126)) + testJson(20281.toShort, ShortType, JLong(20281)) + testJson(-8752.toShort, ShortType, JLong(-8752)) + testJson(1078231987, IntegerType, JLong(1078231987)) + testJson(-10, IntegerType, JLong(-10)) + testJson(139289832109874199L, LongType, JLong(139289832109874199L)) + testJson(-7873748239973488L, LongType, JLong(-7873748239973488L)) + testJson(10.232e10f, FloatType, JDouble(10.232e10f)) + testJson(9.7e-13f, FloatType, JDouble(9.7e-13f)) + testJson(3.891e98d, DoubleType, JDouble(3.891e98d)) + testJson(-7.8e5d, DoubleType, JDouble(-7.8e5d)) + testJson(BigDecimal("1092.88"), DecimalType(10, 2), JDecimal(BigDecimal("1092.88"))) + testJson(Decimal("782.0003"), DecimalType(7, 4), JDecimal(BigDecimal("782.0003"))) + testJson(new java.math.BigDecimal("-77.89"), DecimalType(4, 2), JDecimal(BigDecimal("-77.89"))) + testJson("hello world", StringType, JString("hello world")) + testJson("BinaryType", Array('a'.toByte, 'b'.toByte), BinaryType, JString("YWI=")) + testJson(Date.valueOf("2019-04-22"), DateType, JString("2019-04-22")) + testJson(LocalDate.of(2018, 5, 14), DateType, JString("2018-05-14")) + testJson( + Timestamp.valueOf("2017-01-06 10:22:03.00"), + TimestampType, + JString("2017-01-06 10:22:03")) + testJson( + Timestamp.valueOf("2017-05-30 10:22:03.00").toInstant, + TimestampType, + JString("2017-05-30 10:22:03")) + + // Complex types + testJson( + "ArrayType(LongType,true)", + Array(1L, null, 77L), + ArrayType(LongType, containsNull = true), + JArray(JLong(1L) :: JNull :: JLong(77L) :: Nil)) + + testJson( + Seq(1, -2, 3), + ArrayType(IntegerType, containsNull = false), + JArray(JLong(1) :: JLong(-2) :: JLong(3) :: Nil)) + + testJson( + Map("a" -> "b", "c" -> "d", "e" -> null), + MapType(StringType, StringType, valueContainsNull = true), + JObject("a" -> JString("b"), "c" -> JString("d"), "e" -> JNull)) + + testJson( + Map(1 -> "b", 2 -> "d", 3 -> null), + MapType(IntegerType, StringType, valueContainsNull = true), + JArray( + JObject("key" -> JLong(1), "value" -> JString("b")) :: + JObject("key" -> JLong(2), "value" -> JString("d")) :: + JObject("key" -> JLong(3), "value" -> JNull) :: Nil)) + + testJson( + new GenericRowWithSchema(Array("1", 2), schema), + schema, + JObject("c1" -> JString("1"), "c2" -> JLong(2))) + + testJson( + "UDT", + new ExamplePoint(3.4d, 8.98d), + new ExamplePointUDT, + JArray(JDouble(3.4d) :: JDouble(8.98d) :: Nil)) + + test("no schema") { + val e = intercept[IllegalArgumentException] { + Row("a").jsonValue + } + assert(e.getMessage.contains("requires a non-null schema")) + } + + test("unsupported type") { + val e = intercept[IllegalArgumentException] { + val row = new GenericRowWithSchema( + Array((1, 2)), + new StructType().add("a", ObjectType(classOf[(Int, Int)]))) + row.jsonValue + } + assert(e.getMessage.contains("Failed to convert value")) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 39228102682b9..5c69ef95ffe17 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -27,7 +27,7 @@ class DistributionSuite extends SparkFunSuite { protected def checkSatisfied( inputPartitioning: Partitioning, requiredDistribution: Distribution, - satisfied: Boolean) { + satisfied: Boolean): Unit = { if (inputPartitioning.satisfies(requiredDistribution) != satisfied) { fail( s""" 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 f0356f5a42d67..3dabbca9deeee 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 @@ -532,7 +532,7 @@ class AnalysisErrorSuite extends AnalysisTest { Seq(a, Alias(InSubquery(Seq(a), ListQuery(LocalRelation(b))), "c")()), LocalRelation(a)) assertAnalysisError(plan, "Predicate sub-queries can only be used" + - " in Filter/DeleteFromTable" :: Nil) + " in Filter" :: Nil) } test("PredicateSubQuery is used is a nested condition") { 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 eade9b6112fe4..c01dea96fe2de 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 @@ -21,7 +21,7 @@ import java.net.URI import java.util.Locale import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, LessThanOrEqual, Literal} +import org.apache.spark.sql.catalyst.expressions.{Alias, AnsiCast, AttributeReference, Cast, Expression, LessThanOrEqual, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy @@ -77,57 +77,30 @@ class V2OverwriteByExpressionANSIAnalysisSuite extends DataSourceV2ANSIAnalysisS } 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) + testResolvedOverwriteByExpression() } test("delete expression is not resolved using query fields") { - val xRequiredTable = TestRelation(StructType(Seq( - StructField("x", DoubleType, nullable = false), - StructField("y", DoubleType))).toAttributes) + testNotResolvedOverwriteByExpression() + } +} - val query = TestRelation(StructType(Seq( - StructField("a", DoubleType, nullable = false), - StructField("b", DoubleType))).toAttributes) +class V2OverwriteByExpressionStrictAnalysisSuite extends DataSourceV2StrictAnalysisSuite { + override def byName(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + OverwriteByExpression.byName(table, query, Literal(true)) + } - // 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))) + override def byPosition(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + OverwriteByExpression.byPosition(table, query, Literal(true)) + } - assertNotResolved(parsedPlan) - assertAnalysisError(parsedPlan, Seq("cannot resolve", "`a`", "given input columns", "x, y")) + test("delete expression is resolved using table fields") { + testResolvedOverwriteByExpression() } -} -class V2OverwriteByExpressionStrictAnalysisSuite extends V2OverwriteByExpressionANSIAnalysisSuite { - override def getSQLConf(caseSensitive: Boolean): SQLConf = - super.getSQLConf(caseSensitive) - .copy(SQLConf.STORE_ASSIGNMENT_POLICY -> StoreAssignmentPolicy.STRICT) + test("delete expression is not resolved using query fields") { + testNotResolvedOverwriteByExpression() + } } case class TestRelation(output: Seq[AttributeReference]) extends LeafNode with NamedRelation { @@ -144,6 +117,19 @@ abstract class DataSourceV2ANSIAnalysisSuite extends DataSourceV2AnalysisBaseSui override def getSQLConf(caseSensitive: Boolean): SQLConf = super.getSQLConf(caseSensitive) .copy(SQLConf.STORE_ASSIGNMENT_POLICY -> StoreAssignmentPolicy.ANSI) + + + // For Ansi store assignment policy, expression `AnsiCast` is used instead of `Cast`. + override def checkAnalysis( + inputPlan: LogicalPlan, + expectedPlan: LogicalPlan, + caseSensitive: Boolean): Unit = { + val expectedPlanWithAnsiCast = expectedPlan transformAllExpressions { + case c: Cast => AnsiCast(c.child, c.dataType, c.timeZoneId) + case other => other + } + super.checkAnalysis(inputPlan, expectedPlanWithAnsiCast, caseSensitive) + } } abstract class DataSourceV2StrictAnalysisSuite extends DataSourceV2AnalysisBaseSuite { @@ -571,4 +557,51 @@ abstract class DataSourceV2AnalysisBaseSuite extends AnalysisTest { def toLower(attr: AttributeReference): AttributeReference = { AttributeReference(attr.name.toLowerCase(Locale.ROOT), attr.dataType)(attr.exprId) } + + protected def testResolvedOverwriteByExpression(): Unit = { + 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) + } + + protected def testNotResolvedOverwriteByExpression(): Unit = { + 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")) + } } 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 4440ac9e281c4..ed11bce5d12b4 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 @@ -146,7 +146,6 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { 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") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 74a8590b5eefe..5aa80e1a9bd7f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{InSubquery, ListQuery} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, Project} +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical._ /** * Unit tests for [[ResolveSubquery]]. @@ -29,8 +30,10 @@ class ResolveSubquerySuite extends AnalysisTest { val a = 'a.int val b = 'b.int + val c = 'c.int val t1 = LocalRelation(a) val t2 = LocalRelation(b) + val t3 = LocalRelation(c) test("SPARK-17251 Improve `OuterReference` to be `NamedExpression`") { val expr = Filter( @@ -41,4 +44,13 @@ class ResolveSubquerySuite extends AnalysisTest { assert(m.contains( "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses")) } + + test("SPARK-29145 Support subquery in join condition") { + val expr = Join(t1, + t2, + Inner, + Some(InSubquery(Seq(a), ListQuery(Project(Seq(UnresolvedAttribute("c")), t3)))), + JoinHint.NONE) + assertAnalysisSuccess(expr) + } } 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 50c38145ae21d..4f9e4ec0201dd 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 @@ -690,7 +690,8 @@ class TypeCoercionSuite extends AnalysisTest { Some(new StructType().add("a", StringType))) } - private def ruleTest(rule: Rule[LogicalPlan], initial: Expression, transformed: Expression) { + private def ruleTest(rule: Rule[LogicalPlan], + initial: Expression, transformed: Expression): Unit = { ruleTest(Seq(rule), initial, transformed) } @@ -1429,7 +1430,13 @@ class TypeCoercionSuite extends AnalysisTest { 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)) + ruleTest(dateTimeOperations, Subtract(date, date), SubtractDates(date, date)) + ruleTest(dateTimeOperations, Subtract(timestamp, timestamp), + SubtractTimestamps(timestamp, timestamp)) + ruleTest(dateTimeOperations, Subtract(timestamp, date), + SubtractTimestamps(timestamp, Cast(date, TimestampType))) + ruleTest(dateTimeOperations, Subtract(date, timestamp), + SubtractTimestamps(Cast(date, TimestampType), timestamp)) } /** @@ -1483,15 +1490,15 @@ class TypeCoercionSuite extends AnalysisTest { 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) { + Seq(SQLConf.Dialect.SPARK, SQLConf.Dialect.POSTGRESQL).foreach { dialect => + withSQLConf(SQLConf.DIALECT.key -> dialect.toString) { + val result1 = if (dialect == SQLConf.Dialect.POSTGRESQL) { IntegralDivide(1L, 1L) } else { Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) } ruleTest(rules, Divide(1L, 1L), result1) - val result2 = if (preferIntegralDivision) { + val result2 = if (dialect == SQLConf.Dialect.POSTGRESQL) { IntegralDivide(1, Cast(1, ShortType)) } else { Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) 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 0fe646edb340e..3ec6fdeedd4b8 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 @@ -643,6 +643,153 @@ class UnsupportedOperationsSuite extends SparkFunSuite { null, new TestStreamingRelationV2(attribute)), OutputMode.Append()) + // streaming aggregation + { + assertPassOnGlobalWatermarkLimit( + "single streaming aggregation in Append mode", + streamRelation.groupBy("a")(count("*")), + OutputMode.Append()) + + assertFailOnGlobalWatermarkLimit( + "chained streaming aggregations in Append mode", + streamRelation.groupBy("a")(count("*")).groupBy()(count("*")), + OutputMode.Append()) + + Seq(Inner, LeftOuter, RightOuter).foreach { joinType => + val plan = streamRelation.join(streamRelation.groupBy("a")(count("*")), joinType = joinType) + assertFailOnGlobalWatermarkLimit( + s"$joinType join after streaming aggregation in Append mode", + streamRelation.join(streamRelation.groupBy("a")(count("*")), joinType = joinType), + OutputMode.Append()) + } + + assertFailOnGlobalWatermarkLimit( + "deduplicate after streaming aggregation in Append mode", + Deduplicate(Seq(attribute), streamRelation.groupBy("a")(count("*"))), + OutputMode.Append()) + + assertFailOnGlobalWatermarkLimit( + "FlatMapGroupsWithState after streaming aggregation in Append mode", + FlatMapGroupsWithState( + null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, + streamRelation.groupBy("a")(count("*"))), + OutputMode.Append()) + } + + // stream-stream join + // stream-stream inner join doesn't emit late rows, whereas outer joins could + Seq((Inner, false), (LeftOuter, true), (RightOuter, true)).map { case (joinType, expectFailure) => + assertPassOnGlobalWatermarkLimit( + s"single $joinType join in Append mode", + streamRelation.join(streamRelation, joinType = RightOuter, + condition = Some(attributeWithWatermark === attribute)), + OutputMode.Append()) + + testGlobalWatermarkLimit( + s"streaming aggregation after stream-stream $joinType join in Append mode", + streamRelation.join(streamRelation, joinType = joinType, + condition = Some(attributeWithWatermark === attribute)) + .groupBy("a")(count("*")), + OutputMode.Append(), + expectFailure = expectFailure) + + Seq(Inner, LeftOuter, RightOuter).map { joinType2 => + testGlobalWatermarkLimit( + s"streaming-stream $joinType2 after stream-stream $joinType join in Append mode", + streamRelation.join( + streamRelation.join(streamRelation, joinType = joinType, + condition = Some(attributeWithWatermark === attribute)), + joinType = joinType2, + condition = Some(attributeWithWatermark === attribute)), + OutputMode.Append(), + expectFailure = expectFailure) + } + + testGlobalWatermarkLimit( + s"FlatMapGroupsWithState after stream-stream $joinType join in Append mode", + FlatMapGroupsWithState( + null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, + streamRelation.join(streamRelation, joinType = joinType, + condition = Some(attributeWithWatermark === attribute))), + OutputMode.Append(), + expectFailure = expectFailure) + + testGlobalWatermarkLimit( + s"deduplicate after stream-stream $joinType join in Append mode", + Deduplicate(Seq(attribute), streamRelation.join(streamRelation, joinType = joinType, + condition = Some(attributeWithWatermark === attribute))), + OutputMode.Append(), + expectFailure = expectFailure) + } + + // FlatMapGroupsWithState + { + assertPassOnGlobalWatermarkLimit( + "single FlatMapGroupsWithState in Append mode", + FlatMapGroupsWithState( + null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, streamRelation), + OutputMode.Append()) + + assertFailOnGlobalWatermarkLimit( + "streaming aggregation after FlatMapGroupsWithState in Append mode", + FlatMapGroupsWithState( + null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, streamRelation).groupBy("*")(count("*")), + OutputMode.Append()) + + Seq(Inner, LeftOuter, RightOuter).map { joinType => + assertFailOnGlobalWatermarkLimit( + s"stream-stream $joinType after FlatMapGroupsWithState in Append mode", + streamRelation.join( + FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, streamRelation), joinType = joinType, + condition = Some(attributeWithWatermark === attribute)), + OutputMode.Append()) + } + + assertFailOnGlobalWatermarkLimit( + "FlatMapGroupsWithState after FlatMapGroupsWithState in Append mode", + FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, + FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, streamRelation)), + OutputMode.Append()) + + assertFailOnGlobalWatermarkLimit( + s"deduplicate after FlatMapGroupsWithState in Append mode", + Deduplicate(Seq(attribute), + FlatMapGroupsWithState(null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, streamRelation)), + OutputMode.Append()) + } + + // deduplicate + { + assertPassOnGlobalWatermarkLimit( + "streaming aggregation after deduplicate in Append mode", + Deduplicate(Seq(attribute), streamRelation).groupBy("a")(count("*")), + OutputMode.Append()) + + Seq(Inner, LeftOuter, RightOuter).map { joinType => + assertPassOnGlobalWatermarkLimit( + s"$joinType join after deduplicate in Append mode", + streamRelation.join(Deduplicate(Seq(attribute), streamRelation), joinType = joinType, + condition = Some(attributeWithWatermark === attribute)), + OutputMode.Append()) + } + + assertPassOnGlobalWatermarkLimit( + "FlatMapGroupsWithState after deduplicate in Append mode", + FlatMapGroupsWithState( + null, att, att, Seq(att), Seq(att), att, null, Append, + isMapGroupsWithState = false, null, + Deduplicate(Seq(attribute), streamRelation)), + OutputMode.Append()) + } + /* ======================================================================================= TESTING FUNCTIONS @@ -839,6 +986,40 @@ class UnsupportedOperationsSuite extends SparkFunSuite { } } + + def assertPassOnGlobalWatermarkLimit( + testNamePostfix: String, + plan: LogicalPlan, + outputMode: OutputMode): Unit = { + testGlobalWatermarkLimit(testNamePostfix, plan, outputMode, expectFailure = false) + } + + def assertFailOnGlobalWatermarkLimit( + testNamePostfix: String, + plan: LogicalPlan, + outputMode: OutputMode): Unit = { + testGlobalWatermarkLimit(testNamePostfix, plan, outputMode, expectFailure = true) + } + + def testGlobalWatermarkLimit( + testNamePostfix: String, + plan: LogicalPlan, + outputMode: OutputMode, + expectFailure: Boolean): Unit = { + test(s"Global watermark limit - $testNamePostfix") { + if (expectFailure) { + val e = intercept[AnalysisException] { + UnsupportedOperationChecker.checkStreamingQueryGlobalWatermarkLimit( + wrapInStreaming(plan), outputMode, failWhenDetected = true) + } + assert(e.message.contains("Detected pattern of possible 'correctness' issue")) + } else { + UnsupportedOperationChecker.checkStreamingQueryGlobalWatermarkLimit( + wrapInStreaming(plan), outputMode, failWhenDetected = true) + } + } + } + /** * Test whether the body of code will fail. If it does fail, then check if it has expected * messages. 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 59fb941f41f7e..f334ba5690af0 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 @@ -87,7 +87,7 @@ abstract class SessionCatalogSuite extends AnalysisTest { } } - def testInvalidName(func: (String) => Unit) { + def testInvalidName(func: (String) => Unit): Unit = { // scalastyle:off // non ascii characters are not allowed in the source code, so we disable the scalastyle. val name = "ç –" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index 838ac42184fa5..4696a578f2f91 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.csv +import org.scalatest.prop.TableDrivenPropertyChecks._ + import org.apache.spark.SparkFunSuite class CSVExprUtilsSuite extends SparkFunSuite { @@ -58,4 +60,40 @@ class CSVExprUtilsSuite extends SparkFunSuite { } assert(exception.getMessage.contains("Delimiter cannot be empty string")) } + + val testCases = Table( + ("input", "separatorStr", "expectedErrorMsg"), + // normal tab + ("""\t""", Some("\t"), None), + // backslash, then tab + ("""\\t""", Some("""\t"""), None), + // invalid special character (dot) + ("""\.""", None, Some("Unsupported special character for delimiter")), + // backslash, then dot + ("""\\.""", Some("""\."""), None), + // nothing special, just straight conversion + ("""foo""", Some("foo"), None), + // tab in the middle of some other letters + ("""ba\tr""", Some("ba\tr"), None), + // null character, expressed in Unicode literal syntax + ("""\u0000""", Some("\u0000"), None), + // and specified directly + ("\0", Some("\u0000"), None) + ) + + test("should correctly produce separator strings, or exceptions, from input") { + forAll(testCases) { (input, separatorStr, expectedErrorMsg) => + try { + val separator = CSVExprUtils.toDelimiterStr(input) + assert(separatorStr.isDefined) + assert(expectedErrorMsg.isEmpty) + assert(separator.equals(separatorStr.get)) + } catch { + case e: IllegalArgumentException => + assert(separatorStr.isEmpty) + assert(expectedErrorMsg.isDefined) + assert(e.getMessage.contains(expectedErrorMsg.get)) + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala index 24d909ed99b93..909d0a12b2626 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala @@ -101,13 +101,13 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { assert( inferSchema.mergeRowTypes(Array(StringType), - Array(DoubleType)).deep == Array(StringType).deep) + Array(DoubleType)).sameElements(Array(StringType))) assert( inferSchema.mergeRowTypes(Array(IntegerType), - Array(LongType)).deep == Array(LongType).deep) + Array(LongType)).sameElements(Array(LongType))) assert( inferSchema.mergeRowTypes(Array(DoubleType), - Array(LongType)).deep == Array(DoubleType).deep) + Array(LongType)).sameElements(Array(DoubleType))) } test("Null fields are handled properly when a nullValue is specified") { @@ -133,7 +133,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { val inferSchema = new CSVInferSchema(options) val mergedNullTypes = inferSchema.mergeRowTypes(Array(NullType), Array(NullType)) - assert(mergedNullTypes.deep == Array(NullType).deep) + assert(mergedNullTypes.sameElements(Array(NullType))) } test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { 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 d2acfb9686700..c1f1be3b30e4b 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 @@ -553,7 +553,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes r } - private def testAndVerifyNotLeakingReflectionObjects(testName: String)(testFun: => Any) { + private def testAndVerifyNotLeakingReflectionObjects(testName: String)(testFun: => Any): Unit = { test(testName) { verifyNotLeakingReflectionObjects(testFun) } 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 ffb14e2838687..fc7a0d3af4e28 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 @@ -21,6 +21,8 @@ import java.sql.{Date, Timestamp} import java.util.{Calendar, TimeZone} import java.util.concurrent.TimeUnit._ +import scala.collection.parallel.immutable.ParVector + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow @@ -33,24 +35,19 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -/** - * Test suite for data type casting expression [[Cast]]. - */ -class CastSuite extends SparkFunSuite with ExpressionEvalHelper { +abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { - private def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): Cast = { - v match { - case lit: Expression => Cast(lit, targetType, timeZoneId) - case _ => Cast(Literal(v), targetType, timeZoneId) - } - } + // Whether it is required to set SQLConf.ANSI_ENABLED as true for testing numeric overflow. + protected def requiredAnsiEnabledForOverflowTestCases: Boolean + + protected def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): CastBase // expected cannot be null - private def checkCast(v: Any, expected: Any): Unit = { + protected def checkCast(v: Any, expected: Any): Unit = { checkEvaluation(cast(v, Literal(expected).dataType), expected) } - private def checkNullCast(from: DataType, to: DataType): Unit = { + protected def checkNullCast(from: DataType, to: DataType): Unit = { checkEvaluation(cast(Literal.create(null, from), to, Option("GMT")), null) } @@ -113,7 +110,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to timestamp") { - ALL_TIMEZONES.par.foreach { tz => + new ParVector(ALL_TIMEZONES.toVector).foreach { tz => def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)), expected) } @@ -207,43 +204,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("cast from int") { - checkCast(0, false) - checkCast(1, true) - checkCast(-5, true) - checkCast(1, 1.toByte) - checkCast(1, 1.toShort) - checkCast(1, 1) - checkCast(1, 1.toLong) - checkCast(1, 1.0f) - checkCast(1, 1.0) - checkCast(123, "123") - - checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) - checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) - checkEvaluation(cast(123, DecimalType(3, 1)), null) - checkEvaluation(cast(123, DecimalType(2, 0)), null) - } - - test("cast from long") { - checkCast(0L, false) - checkCast(1L, true) - checkCast(-5L, true) - checkCast(1L, 1.toByte) - checkCast(1L, 1.toShort) - checkCast(1L, 1) - checkCast(1L, 1.toLong) - checkCast(1L, 1.0f) - checkCast(1L, 1.0) - checkCast(123L, "123") - - checkEvaluation(cast(123L, DecimalType.USER_DEFAULT), Decimal(123)) - checkEvaluation(cast(123L, DecimalType(3, 0)), Decimal(123)) - checkEvaluation(cast(123L, DecimalType(3, 1)), null) - - checkEvaluation(cast(123L, DecimalType(2, 0)), null) - } - test("cast from boolean") { checkEvaluation(cast(true, IntegerType), 1) checkEvaluation(cast(false, IntegerType), 0) @@ -253,17 +213,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(cast(0, BooleanType), IntegerType), 0) } - test("cast from int 2") { - checkEvaluation(cast(1, LongType), 1.toLong) - checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) - checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) - - checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) - checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) - checkEvaluation(cast(123, DecimalType(3, 1)), null) - checkEvaluation(cast(123, DecimalType(2, 0)), null) - } - test("cast from float") { checkCast(0.0f, false) checkCast(0.5f, true) @@ -401,101 +350,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkCast(Decimal(1.5), "1.5") } - test("casting to fixed-precision decimals") { - assert(cast(123, DecimalType.USER_DEFAULT).nullable === false) - 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) - 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) - 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) - assert(cast(Decimal(10.03), DecimalType(4, 1)).nullable === false) - 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) - assert(cast(Decimal("1003"), DecimalType(4, -1)).nullable === false) - 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) - - - checkEvaluation(cast(10.03, DecimalType.SYSTEM_DEFAULT), Decimal(10.03)) - checkEvaluation(cast(10.03, DecimalType(4, 2)), Decimal(10.03)) - checkEvaluation(cast(10.03, DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(cast(10.03, DecimalType(2, 0)), Decimal(10)) - checkEvaluation(cast(10.03, DecimalType(1, 0)), null) - checkEvaluation(cast(10.03, DecimalType(2, 1)), null) - checkEvaluation(cast(10.03, DecimalType(3, 2)), null) - checkEvaluation(cast(Decimal(10.03), DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(cast(Decimal(10.03), DecimalType(3, 2)), null) - - checkEvaluation(cast(10.05, DecimalType.SYSTEM_DEFAULT), Decimal(10.05)) - checkEvaluation(cast(10.05, DecimalType(4, 2)), Decimal(10.05)) - checkEvaluation(cast(10.05, DecimalType(3, 1)), Decimal(10.1)) - checkEvaluation(cast(10.05, DecimalType(2, 0)), Decimal(10)) - checkEvaluation(cast(10.05, DecimalType(1, 0)), null) - checkEvaluation(cast(10.05, DecimalType(2, 1)), null) - checkEvaluation(cast(10.05, DecimalType(3, 2)), null) - checkEvaluation(cast(Decimal(10.05), DecimalType(3, 1)), Decimal(10.1)) - checkEvaluation(cast(Decimal(10.05), DecimalType(3, 2)), null) - - checkEvaluation(cast(9.95, DecimalType(3, 2)), Decimal(9.95)) - checkEvaluation(cast(9.95, DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(cast(9.95, DecimalType(2, 0)), Decimal(10)) - checkEvaluation(cast(9.95, DecimalType(2, 1)), null) - checkEvaluation(cast(9.95, DecimalType(1, 0)), null) - checkEvaluation(cast(Decimal(9.95), DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(cast(Decimal(9.95), DecimalType(1, 0)), null) - - checkEvaluation(cast(-9.95, DecimalType(3, 2)), Decimal(-9.95)) - checkEvaluation(cast(-9.95, DecimalType(3, 1)), Decimal(-10.0)) - checkEvaluation(cast(-9.95, DecimalType(2, 0)), Decimal(-10)) - checkEvaluation(cast(-9.95, DecimalType(2, 1)), null) - checkEvaluation(cast(-9.95, DecimalType(1, 0)), null) - checkEvaluation(cast(Decimal(-9.95), DecimalType(3, 1)), Decimal(-10.0)) - checkEvaluation(cast(Decimal(-9.95), DecimalType(1, 0)), null) - - checkEvaluation(cast(Decimal("1003"), DecimalType.SYSTEM_DEFAULT), Decimal(1003)) - checkEvaluation(cast(Decimal("1003"), DecimalType(4, 0)), Decimal(1003)) - checkEvaluation(cast(Decimal("1003"), DecimalType(3, -1)), Decimal(1000)) - checkEvaluation(cast(Decimal("1003"), DecimalType(2, -2)), Decimal(1000)) - checkEvaluation(cast(Decimal("1003"), DecimalType(1, -2)), null) - checkEvaluation(cast(Decimal("1003"), DecimalType(2, -1)), null) - checkEvaluation(cast(Decimal("1003"), DecimalType(3, 0)), null) - - checkEvaluation(cast(Decimal("995"), DecimalType(3, 0)), Decimal(995)) - checkEvaluation(cast(Decimal("995"), DecimalType(3, -1)), Decimal(1000)) - checkEvaluation(cast(Decimal("995"), DecimalType(2, -2)), Decimal(1000)) - checkEvaluation(cast(Decimal("995"), DecimalType(2, -1)), null) - checkEvaluation(cast(Decimal("995"), DecimalType(1, -2)), null) - - checkEvaluation(cast(Double.NaN, DecimalType.SYSTEM_DEFAULT), null) - checkEvaluation(cast(1.0 / 0.0, DecimalType.SYSTEM_DEFAULT), null) - checkEvaluation(cast(Float.NaN, DecimalType.SYSTEM_DEFAULT), null) - checkEvaluation(cast(1.0f / 0.0f, DecimalType.SYSTEM_DEFAULT), null) - - checkEvaluation(cast(Double.NaN, DecimalType(2, 1)), null) - checkEvaluation(cast(1.0 / 0.0, DecimalType(2, 1)), null) - checkEvaluation(cast(Float.NaN, DecimalType(2, 1)), null) - checkEvaluation(cast(1.0f / 0.0f, DecimalType(2, 1)), null) - - checkEvaluation(cast(true, DecimalType(2, 1)), Decimal(1)) - checkEvaluation(cast(true, DecimalType(1, 1)), null) - } - test("cast from date") { val d = Date.valueOf("1970-01-01") checkEvaluation(cast(d, ShortType), null) @@ -816,38 +670,30 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { new CalendarInterval(15, -3 * CalendarInterval.MICROS_PER_DAY), CalendarIntervalType), StringType), "interval 1 years 3 months -3 days") + checkEvaluation(Cast(Literal("INTERVAL 1 Second 1 microsecond"), CalendarIntervalType), + new CalendarInterval(0, 1000001)) + checkEvaluation(Cast(Literal("1 MONTH 1 Microsecond"), CalendarIntervalType), + new CalendarInterval(1, 1)) } test("cast string to boolean") { - checkCast("true", true) - checkCast("tru", true) - checkCast("tr", true) checkCast("t", true) + checkCast("true", true) checkCast("tRUe", true) - checkCast(" tRue ", true) - checkCast(" tRu ", true) - checkCast("yes", true) - checkCast("ye", true) checkCast("y", true) + checkCast("yes", true) checkCast("1", true) - checkCast("on", true) - checkCast("false", false) - checkCast("fals", false) - checkCast("fal", false) - checkCast("fa", false) checkCast("f", false) - checkCast(" fAlse ", false) - checkCast(" fAls ", false) - checkCast(" FAlsE ", false) - checkCast("no", false) + checkCast("false", false) + checkCast("FAlsE", false) checkCast("n", false) + checkCast("no", false) checkCast("0", false) - checkCast("off", false) - checkCast("of", false) - checkEvaluation(cast("o", BooleanType), null) checkEvaluation(cast("abc", BooleanType), null) + checkEvaluation(cast("tru", BooleanType), null) + checkEvaluation(cast("fla", BooleanType), null) checkEvaluation(cast("", BooleanType), null) } @@ -1043,24 +889,17 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { - checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) - checkEvaluation( - Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) - checkEvaluation(Cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), null) - checkEvaluation(Cast(Literal(134.12), DecimalType(3, 2)), null) - } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + test("Throw exception on casting out-of-range value to decimal type") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { checkExceptionInExpression[ArithmeticException]( - Cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") + cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( - Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), + cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( - Cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), "cannot be represented") + cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( - Cast(Literal(134.12), DecimalType(3, 2)), "cannot be represented") + cast(Literal(134.12), DecimalType(3, 2)), "cannot be represented") } } @@ -1116,8 +955,8 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("Cast to byte with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + test("Throw exception on casting out-of-range value to byte type") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -1141,8 +980,8 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("Cast to short with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + test("Throw exception on casting out-of-range value to short type") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1166,8 +1005,8 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("Cast to int with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + test("Throw exception on casting out-of-range value to int type") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1183,8 +1022,8 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("Cast to long with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + test("Throw exception on casting out-of-range value to long type") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => @@ -1201,3 +1040,187 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + +/** + * Test suite for data type casting expression [[Cast]]. + */ +class CastSuite extends CastSuiteBase { + // It is required to set SQLConf.ANSI_ENABLED as true for testing numeric overflow. + override protected def requiredAnsiEnabledForOverflowTestCases: Boolean = true + + override def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): CastBase = { + v match { + case lit: Expression => Cast(lit, targetType, timeZoneId) + case _ => Cast(Literal(v), targetType, timeZoneId) + } + } + + + test("cast from int") { + checkCast(0, false) + checkCast(1, true) + checkCast(-5, true) + checkCast(1, 1.toByte) + checkCast(1, 1.toShort) + checkCast(1, 1) + checkCast(1, 1.toLong) + checkCast(1, 1.0f) + checkCast(1, 1.0) + checkCast(123, "123") + + checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 1)), null) + checkEvaluation(cast(123, DecimalType(2, 0)), null) + } + + test("cast from long") { + checkCast(0L, false) + checkCast(1L, true) + checkCast(-5L, true) + checkCast(1L, 1.toByte) + checkCast(1L, 1.toShort) + checkCast(1L, 1) + checkCast(1L, 1.toLong) + checkCast(1L, 1.0f) + checkCast(1L, 1.0) + checkCast(123L, "123") + + checkEvaluation(cast(123L, DecimalType.USER_DEFAULT), Decimal(123)) + checkEvaluation(cast(123L, DecimalType(3, 0)), Decimal(123)) + checkEvaluation(cast(123L, DecimalType(3, 1)), null) + + checkEvaluation(cast(123L, DecimalType(2, 0)), null) + } + + test("cast from int 2") { + checkEvaluation(cast(1, LongType), 1.toLong) + checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) + checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + + checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 1)), null) + checkEvaluation(cast(123, DecimalType(2, 0)), null) + } + + test("casting to fixed-precision decimals") { + assert(cast(123, DecimalType.USER_DEFAULT).nullable === false) + 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) + 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) + 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) + assert(cast(Decimal(10.03), DecimalType(4, 1)).nullable === false) + 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) + assert(cast(Decimal("1003"), DecimalType(4, -1)).nullable === false) + 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) + + + checkEvaluation(cast(10.03, DecimalType.SYSTEM_DEFAULT), Decimal(10.03)) + checkEvaluation(cast(10.03, DecimalType(4, 2)), Decimal(10.03)) + checkEvaluation(cast(10.03, DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(10.03, DecimalType(2, 0)), Decimal(10)) + checkEvaluation(cast(10.03, DecimalType(1, 0)), null) + checkEvaluation(cast(10.03, DecimalType(2, 1)), null) + checkEvaluation(cast(10.03, DecimalType(3, 2)), null) + checkEvaluation(cast(Decimal(10.03), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(Decimal(10.03), DecimalType(3, 2)), null) + + checkEvaluation(cast(10.05, DecimalType.SYSTEM_DEFAULT), Decimal(10.05)) + checkEvaluation(cast(10.05, DecimalType(4, 2)), Decimal(10.05)) + checkEvaluation(cast(10.05, DecimalType(3, 1)), Decimal(10.1)) + checkEvaluation(cast(10.05, DecimalType(2, 0)), Decimal(10)) + checkEvaluation(cast(10.05, DecimalType(1, 0)), null) + checkEvaluation(cast(10.05, DecimalType(2, 1)), null) + checkEvaluation(cast(10.05, DecimalType(3, 2)), null) + checkEvaluation(cast(Decimal(10.05), DecimalType(3, 1)), Decimal(10.1)) + checkEvaluation(cast(Decimal(10.05), DecimalType(3, 2)), null) + + checkEvaluation(cast(9.95, DecimalType(3, 2)), Decimal(9.95)) + checkEvaluation(cast(9.95, DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(9.95, DecimalType(2, 0)), Decimal(10)) + checkEvaluation(cast(9.95, DecimalType(2, 1)), null) + checkEvaluation(cast(9.95, DecimalType(1, 0)), null) + checkEvaluation(cast(Decimal(9.95), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(Decimal(9.95), DecimalType(1, 0)), null) + + checkEvaluation(cast(-9.95, DecimalType(3, 2)), Decimal(-9.95)) + checkEvaluation(cast(-9.95, DecimalType(3, 1)), Decimal(-10.0)) + checkEvaluation(cast(-9.95, DecimalType(2, 0)), Decimal(-10)) + checkEvaluation(cast(-9.95, DecimalType(2, 1)), null) + checkEvaluation(cast(-9.95, DecimalType(1, 0)), null) + checkEvaluation(cast(Decimal(-9.95), DecimalType(3, 1)), Decimal(-10.0)) + checkEvaluation(cast(Decimal(-9.95), DecimalType(1, 0)), null) + + checkEvaluation(cast(Decimal("1003"), DecimalType.SYSTEM_DEFAULT), Decimal(1003)) + checkEvaluation(cast(Decimal("1003"), DecimalType(4, 0)), Decimal(1003)) + checkEvaluation(cast(Decimal("1003"), DecimalType(3, -1)), Decimal(1000)) + checkEvaluation(cast(Decimal("1003"), DecimalType(2, -2)), Decimal(1000)) + checkEvaluation(cast(Decimal("1003"), DecimalType(1, -2)), null) + checkEvaluation(cast(Decimal("1003"), DecimalType(2, -1)), null) + checkEvaluation(cast(Decimal("1003"), DecimalType(3, 0)), null) + + checkEvaluation(cast(Decimal("995"), DecimalType(3, 0)), Decimal(995)) + checkEvaluation(cast(Decimal("995"), DecimalType(3, -1)), Decimal(1000)) + checkEvaluation(cast(Decimal("995"), DecimalType(2, -2)), Decimal(1000)) + checkEvaluation(cast(Decimal("995"), DecimalType(2, -1)), null) + checkEvaluation(cast(Decimal("995"), DecimalType(1, -2)), null) + + checkEvaluation(cast(Double.NaN, DecimalType.SYSTEM_DEFAULT), null) + checkEvaluation(cast(1.0 / 0.0, DecimalType.SYSTEM_DEFAULT), null) + checkEvaluation(cast(Float.NaN, DecimalType.SYSTEM_DEFAULT), null) + checkEvaluation(cast(1.0f / 0.0f, DecimalType.SYSTEM_DEFAULT), null) + + checkEvaluation(cast(Double.NaN, DecimalType(2, 1)), null) + checkEvaluation(cast(1.0 / 0.0, DecimalType(2, 1)), null) + checkEvaluation(cast(Float.NaN, DecimalType(2, 1)), null) + checkEvaluation(cast(1.0f / 0.0f, DecimalType(2, 1)), null) + + checkEvaluation(cast(true, DecimalType(2, 1)), Decimal(1)) + checkEvaluation(cast(true, DecimalType(1, 1)), null) + } + + test("SPARK-28470: Cast should honor nullOnOverflow property") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) + checkEvaluation( + Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) + checkEvaluation(Cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), null) + checkEvaluation(Cast(Literal(134.12), DecimalType(3, 2)), null) + } + } +} + +/** + * Test suite for data type casting expression [[AnsiCast]]. + */ +class AnsiCastSuite extends CastSuiteBase { + // It is not required to set SQLConf.ANSI_ENABLED as true for testing numeric overflow. + override protected def requiredAnsiEnabledForOverflowTestCases: Boolean = false + + override def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): CastBase = { + v match { + case lit: Expression => AnsiCast(lit, targetType, timeZoneId) + case _ => AnsiCast(Literal(v), targetType, timeZoneId) + } + } +} 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 603073b40d7aa..e10aa60d52cf8 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 @@ -26,7 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH @@ -720,7 +720,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours"))), + Literal(IntervalUtils.fromString("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -729,7 +729,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:01")), - Literal(CalendarInterval.fromString("interval 12 hours"))), + Literal(IntervalUtils.fromString("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -738,7 +738,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(IntervalUtils.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -747,7 +747,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(IntervalUtils.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -756,7 +756,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -765,7 +765,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(IntervalUtils.fromString("interval 1 month").negate())), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -774,7 +774,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 day").negate())), + Literal(IntervalUtils.fromString("interval 1 month 1 day").negate())), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -783,7 +783,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-31 00:00:00")), Literal(Timestamp.valueOf("2018-04-30 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-31 00:00:00"), Timestamp.valueOf("2018-02-28 00:00:00"), @@ -793,7 +793,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 second"))), + Literal(IntervalUtils.fromString("interval 1 month 1 second"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:01"))) @@ -801,7 +801,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:04:06")), - Literal(CalendarInterval.fromString("interval 1 month 2 minutes 3 seconds"))), + Literal(IntervalUtils.fromString("interval 1 month 2 minutes 3 seconds"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:02:03"), @@ -839,7 +839,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-25 01:30:00")), Literal(Timestamp.valueOf("2018-03-25 03:30:00")), - Literal(CalendarInterval.fromString("interval 30 minutes"))), + Literal(IntervalUtils.fromString("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-03-25 01:30:00"), Timestamp.valueOf("2018-03-25 03:00:00"), @@ -849,7 +849,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-10-28 01:30:00")), Literal(Timestamp.valueOf("2018-10-28 03:30:00")), - Literal(CalendarInterval.fromString("interval 30 minutes"))), + Literal(IntervalUtils.fromString("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-10-28 01:30:00"), noDST(Timestamp.valueOf("2018-10-28 02:00:00")), @@ -866,7 +866,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-01-05")), - Literal(CalendarInterval.fromString("interval 2 days"))), + Literal(IntervalUtils.fromString("interval 2 days"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-01-03"), @@ -875,7 +875,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-03-01")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-02-01"), @@ -884,7 +884,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-31")), Literal(Date.valueOf("2018-04-30")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Date.valueOf("2018-01-31"), Date.valueOf("2018-02-28"), @@ -905,14 +905,14 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-02")), Literal(Date.valueOf("1970-01-01")), - Literal(CalendarInterval.fromString("interval 1 day"))), + Literal(IntervalUtils.fromString("interval 1 day"))), EmptyRow, "sequence boundaries: 1 to 0 by 1") checkExceptionInExpression[IllegalArgumentException]( new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(IntervalUtils.fromString("interval 1 month").negate())), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * CalendarInterval.MICROS_PER_DAY}") } 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 0c4438987cd2a..9039cd6451590 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 @@ -369,7 +369,6 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val b = AttributeReference("b", IntegerType)() checkMetadata(CreateStruct(Seq(a, b))) checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) - checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) } test("StringToMap") { 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 ae3549b4aaf5f..6abadd77bd41a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{LocalDateTime, ZoneId, ZoneOffset} +import java.time.{Instant, LocalDate, LocalDateTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -28,7 +28,7 @@ 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.{DateTimeUtils, IntervalUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.internal.SQLConf @@ -1053,4 +1053,58 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(IsoYear(MakeDate(Literal(2006), Literal(1), Literal(1))), 2005) checkEvaluation(IsoYear(MakeDate(Literal(2006), Literal(1), Literal(2))), 2006) } + + test("extract the seconds part with fraction from timestamps") { + outstandingTimezonesIds.foreach { timezone => + val timestamp = MakeTimestamp(Literal(2019), Literal(8), Literal(10), + Literal(0), Literal(0), Literal(Decimal(10.123456, 8, 6)), + Some(Literal(timezone))) + + checkEvaluation(SecondWithFraction(timestamp), Decimal(10.123456, 8, 6)) + checkEvaluation( + SecondWithFraction(timestamp.copy(sec = Literal(Decimal(59000001, 8, 6)))), + Decimal(59000001, 8, 6)) + checkEvaluation( + SecondWithFraction(timestamp.copy(sec = Literal(Decimal(1, 8, 6)))), + Decimal(0.000001, 8, 6)) + } + } + + test("timestamps difference") { + val end = Instant.parse("2019-10-04T11:04:01.123456Z") + checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), + new CalendarInterval(0, 0)) + checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), + IntervalUtils.fromString("interval 18173 days " + + "11 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) + checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), + IntervalUtils.fromString("interval -18173 days " + + "-11 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) + checkEvaluation( + SubtractTimestamps( + Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), + Literal(Instant.parse("0001-01-01T00:00:00Z"))), + IntervalUtils.fromString("interval 521722 weeks 4 days " + + "23 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) + } + + test("subtract dates") { + val end = LocalDate.of(2019, 10, 5) + checkEvaluation(SubtractDates(Literal(end), Literal(end)), + new CalendarInterval(0, 0)) + checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), + IntervalUtils.fromString("interval 1 days")) + checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), + IntervalUtils.fromString("interval -1 days")) + val epochDate = Literal(LocalDate.ofEpochDay(0)) + checkEvaluation(SubtractDates(Literal(end), epochDate), + IntervalUtils.fromString("interval 49 years 9 months 4 days")) + checkEvaluation(SubtractDates(epochDate, Literal(end)), + IntervalUtils.fromString("interval -49 years -9 months -4 days")) + checkEvaluation( + SubtractDates( + Literal(LocalDate.of(10000, 1, 1)), + Literal(LocalDate.of(1, 1, 1))), + IntervalUtils.fromString("interval 9999 years")) + } } 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 bc1f31b101c6e..6f73c1b0c04fb 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 @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.scalacheck.Gen import org.scalactic.TripleEqualsSupport.Spread import org.scalatest.exceptions.TestFailedException -import org.scalatest.prop.GeneratorDrivenPropertyChecks +import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.JavaSerializer @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils /** * A few helper functions for expression evaluation testing. Mixin this trait to use them. */ -trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBase { +trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestBase { self: SparkFunSuite => protected def create_row(values: Any*): InternalRow = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala similarity index 78% rename from sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index ae701f266bf45..442a23218c010 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -15,19 +15,21 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst +package org.apache.spark.sql.catalyst.expressions import java.time.LocalDateTime -import org.apache.spark.sql.QueryTest +import org.apache.spark.SparkFunSuite 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.sql.types.{FloatType, TimestampType} import org.apache.spark.unsafe.types.CalendarInterval -class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { +class ExpressionSQLBuilderSuite extends SparkFunSuite { + import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._ + protected def checkSQL(e: Expression, expectedSQL: String): Unit = { val actualSQL = e.sql try { @@ -42,6 +44,24 @@ class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { |$cause """.stripMargin) } + + // For literals, check that the SQL evaluates to the same value. Excludes timestamp type which + // currently doesn't have a round-trippable format. + if (e.isInstanceOf[Literal] && e.dataType != TimestampType) { + val roundTrippedValue = parseExpression(actualSQL).eval() + + e match { + // NaNs don't compare equal so we need special checks. + case FloatLiteral(f) if f.isNaN => + assert(roundTrippedValue.isInstanceOf[Float]) + assert(roundTrippedValue.asInstanceOf[Float].isNaN) + case DoubleLiteral(d) if d.isNaN => + assert(roundTrippedValue.isInstanceOf[Double]) + assert(roundTrippedValue.asInstanceOf[Double].isNaN) + case lit: Literal => + assert(lit.value === roundTrippedValue) + } + } } test("literal") { @@ -52,20 +72,24 @@ class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { checkSQL(Literal(2: Short), "2S") checkSQL(Literal(4: Int), "4") checkSQL(Literal(8: Long), "8L") - checkSQL(Literal(1.5F), "CAST(1.5 AS FLOAT)") + checkSQL(Literal(1.5F), "CAST('1.5' AS FLOAT)") checkSQL(Literal(Float.PositiveInfinity), "CAST('Infinity' AS FLOAT)") checkSQL(Literal(Float.NegativeInfinity), "CAST('-Infinity' AS FLOAT)") checkSQL(Literal(Float.NaN), "CAST('NaN' AS FLOAT)") + checkSQL(Literal(Float.MinPositiveValue), "CAST('1.4E-45' AS FLOAT)") checkSQL(Literal(2.5D), "2.5D") checkSQL(Literal(Double.PositiveInfinity), "CAST('Infinity' AS DOUBLE)") checkSQL(Literal(Double.NegativeInfinity), "CAST('-Infinity' AS DOUBLE)") checkSQL(Literal(Double.NaN), "CAST('NaN' AS DOUBLE)") + checkSQL(Literal(Double.MinPositiveValue), "4.9E-324D") checkSQL(Literal(BigDecimal("10.0000000").underlying), "10.0000000BD") checkSQL(Literal(Array(0x01, 0xA3).map(_.toByte)), "X'01A3'") - val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0) + + // Nanos are truncated, but micros should not be + val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0, 987654321) .atZone(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) .toInstant - checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00')") + checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00.987654')") // TODO tests for decimals } 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 f90c98be0b3fd..4b2da73abe562 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 @@ -30,9 +30,9 @@ import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.types.{ArrayType, StructType, _} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val random = new scala.util.Random @@ -252,7 +252,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for CalendarInterval type") { def checkHiveHashForIntervalType(interval: String, expected: Long): Unit = { - checkHiveHash(CalendarInterval.fromString(interval), CalendarIntervalType, expected) + checkHiveHash(IntervalUtils.fromString(interval), CalendarIntervalType, expected) } // ----- MICROSEC ----- 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 b83d03025d21c..4cdee447fa45a 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 @@ -89,6 +89,11 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayFilter(expr, createLambda(et, cn, f)).bind(validateBinding) } + def filter(expr: Expression, f: (Expression, Expression) => Expression): Expression = { + val ArrayType(et, cn) = expr.dataType + ArrayFilter(expr, createLambda(et, cn, IntegerType, false, f)).bind(validateBinding) + } + def transformKeys(expr: Expression, f: (Expression, Expression) => Expression): Expression = { val MapType(kt, vt, vcn) = expr.dataType TransformKeys(expr, createLambda(kt, false, vt, vcn, f)).bind(validateBinding) @@ -218,9 +223,11 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val isEven: Expression => Expression = x => x % 2 === 0 val isNullOrOdd: Expression => Expression = x => x.isNull || x % 2 === 1 + val indexIsEven: (Expression, Expression) => Expression = { case (_, idx) => idx % 2 === 0 } checkEvaluation(filter(ai0, isEven), Seq(2)) checkEvaluation(filter(ai0, isNullOrOdd), Seq(1, 3)) + checkEvaluation(filter(ai0, indexIsEven), Seq(1, 3)) checkEvaluation(filter(ai1, isEven), Seq.empty) checkEvaluation(filter(ai1, isNullOrOdd), Seq(1, null, 3)) checkEvaluation(filter(ain, isEven), null) @@ -234,13 +241,17 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val startsWithA: Expression => Expression = x => x.startsWith("a") checkEvaluation(filter(as0, startsWithA), Seq("a0", "a2")) + checkEvaluation(filter(as0, indexIsEven), Seq("a0", "a2")) checkEvaluation(filter(as1, startsWithA), Seq("a")) + checkEvaluation(filter(as1, indexIsEven), Seq("a", "c")) checkEvaluation(filter(asn, startsWithA), null) val aai = Literal.create(Seq(Seq(1, 2, 3), null, Seq(4, 5)), ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = true)) checkEvaluation(transform(aai, ix => filter(ix, isNullOrOdd)), Seq(Seq(1, 3), null, Seq(5))) + checkEvaluation(transform(aai, ix => filter(ix, indexIsEven)), + Seq(Seq(1, 3), null, Seq(4))) } test("ArrayExists") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala new file mode 100644 index 0000000000000..818ee239dbbf8 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.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.expressions + +import scala.language.implicitConversions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.IntervalUtils +import org.apache.spark.sql.types.Decimal + +class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def interval(s: String): Literal = { + Literal(IntervalUtils.fromString("interval " + s)) + } + + test("millenniums") { + checkEvaluation(ExtractIntervalMillenniums("0 years"), 0) + checkEvaluation(ExtractIntervalMillenniums("9999 years"), 9) + checkEvaluation(ExtractIntervalMillenniums("1000 years"), 1) + checkEvaluation(ExtractIntervalMillenniums("-2000 years"), -2) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalMillenniums("999 years 400 days"), 0) + // Millennium must be taken from years and months + checkEvaluation(ExtractIntervalMillenniums("999 years 12 months"), 1) + checkEvaluation(ExtractIntervalMillenniums("1000 years -1 months"), 0) + } + + test("centuries") { + checkEvaluation(ExtractIntervalCenturies("0 years"), 0) + checkEvaluation(ExtractIntervalCenturies("9999 years"), 99) + checkEvaluation(ExtractIntervalCenturies("1000 years"), 10) + checkEvaluation(ExtractIntervalCenturies("-2000 years"), -20) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalCenturies("99 years 400 days"), 0) + // Century must be taken from years and months + checkEvaluation(ExtractIntervalCenturies("99 years 12 months"), 1) + checkEvaluation(ExtractIntervalCenturies("100 years -1 months"), 0) + } + + test("decades") { + checkEvaluation(ExtractIntervalDecades("0 years"), 0) + checkEvaluation(ExtractIntervalDecades("9999 years"), 999) + checkEvaluation(ExtractIntervalDecades("1000 years"), 100) + checkEvaluation(ExtractIntervalDecades("-2000 years"), -200) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalDecades("9 years 400 days"), 0) + // Decade must be taken from years and months + checkEvaluation(ExtractIntervalDecades("9 years 12 months"), 1) + checkEvaluation(ExtractIntervalDecades("10 years -1 months"), 0) + } + + test("years") { + checkEvaluation(ExtractIntervalYears("0 years"), 0) + checkEvaluation(ExtractIntervalYears("9999 years"), 9999) + checkEvaluation(ExtractIntervalYears("1000 years"), 1000) + checkEvaluation(ExtractIntervalYears("-2000 years"), -2000) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalYears("9 years 400 days"), 9) + // Year must be taken from years and months + checkEvaluation(ExtractIntervalYears("9 years 12 months"), 10) + checkEvaluation(ExtractIntervalYears("10 years -1 months"), 9) + } + + test("quarters") { + checkEvaluation(ExtractIntervalQuarters("0 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("1 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("-1 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("2 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("-2 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("1 years -1 months"), 4.toByte) + checkEvaluation(ExtractIntervalQuarters("-1 years 1 months"), -2.toByte) + checkEvaluation(ExtractIntervalQuarters("2 years 3 months"), 2.toByte) + checkEvaluation(ExtractIntervalQuarters("-2 years -3 months"), 0.toByte) + checkEvaluation(ExtractIntervalQuarters("9999 years"), 1.toByte) + } + + test("months") { + checkEvaluation(ExtractIntervalMonths("0 year"), 0.toByte) + for (m <- -24 to 24) { + checkEvaluation(ExtractIntervalMonths(s"$m months"), (m % 12).toByte) + } + checkEvaluation(ExtractIntervalMonths("1 year 10 months"), 10.toByte) + checkEvaluation(ExtractIntervalMonths("-2 year -10 months"), -10.toByte) + checkEvaluation(ExtractIntervalMonths("9999 years"), 0.toByte) + } + + private val largeInterval: String = "9999 years 11 months " + + "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" + + test("days") { + checkEvaluation(ExtractIntervalDays("0 days"), 0L) + checkEvaluation(ExtractIntervalDays("1 days 100 seconds"), 1L) + checkEvaluation(ExtractIntervalDays("-1 days -100 seconds"), -1L) + checkEvaluation(ExtractIntervalDays("-365 days"), -365L) + checkEvaluation(ExtractIntervalDays("365 days"), 365L) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalDays("100 year 10 months 5 days"), 5L) + checkEvaluation(ExtractIntervalDays(largeInterval), 31L) + } + + test("hours") { + checkEvaluation(ExtractIntervalHours("0 hours"), 0.toByte) + checkEvaluation(ExtractIntervalHours("1 hour"), 1.toByte) + checkEvaluation(ExtractIntervalHours("-1 hour"), -1.toByte) + checkEvaluation(ExtractIntervalHours("23 hours"), 23.toByte) + checkEvaluation(ExtractIntervalHours("-23 hours"), -23.toByte) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalHours("100 year 10 months 10 hours"), 10.toByte) + checkEvaluation(ExtractIntervalHours(largeInterval), 11.toByte) + } + + test("minutes") { + checkEvaluation(ExtractIntervalMinutes("0 minute"), 0.toByte) + checkEvaluation(ExtractIntervalMinutes("1 minute"), 1.toByte) + checkEvaluation(ExtractIntervalMinutes("-1 minute"), -1.toByte) + checkEvaluation(ExtractIntervalMinutes("59 minute"), 59.toByte) + checkEvaluation(ExtractIntervalMinutes("-59 minute"), -59.toByte) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalMinutes("100 year 10 months 10 minutes"), 10.toByte) + checkEvaluation(ExtractIntervalMinutes(largeInterval), 59.toByte) + } + + test("seconds") { + checkEvaluation(ExtractIntervalSeconds("0 second"), Decimal(0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("1 second"), Decimal(1.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("-1 second"), Decimal(-1.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("1 minute 59 second"), Decimal(59.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalSeconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds(largeInterval), Decimal(59.999999, 8, 6)) + checkEvaluation( + ExtractIntervalSeconds("10 seconds 1 milliseconds 1 microseconds"), + Decimal(10001001, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) + } + + test("milliseconds") { + checkEvaluation(ExtractIntervalMilliseconds("0 milliseconds"), Decimal(0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("1 second 999 milliseconds"), + Decimal(1999.0, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("999 milliseconds 1 microsecond"), + Decimal(999.001, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("-1 second -999 milliseconds"), + Decimal(-1999.0, 8, 3)) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalMilliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds(largeInterval), Decimal(59999.999, 8, 3)) + } + + test("microseconds") { + checkEvaluation(ExtractIntervalMicroseconds("0 microseconds"), 0L) + checkEvaluation(ExtractIntervalMicroseconds("1 microseconds"), 1L) + checkEvaluation(ExtractIntervalMicroseconds("-1 microseconds"), -1L) + checkEvaluation(ExtractIntervalMicroseconds("1 second 999 microseconds"), 1000999L) + checkEvaluation(ExtractIntervalMicroseconds("999 milliseconds 1 microseconds"), 999001L) + checkEvaluation(ExtractIntervalMicroseconds("-1 second -999 microseconds"), -1000999L) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalMicroseconds("11 year 1 microseconds"), 1L) + checkEvaluation(ExtractIntervalMicroseconds(largeInterval), 59999999L) + } + + test("epoch") { + checkEvaluation(ExtractIntervalEpoch("0 months"), Decimal(0.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) + checkEvaluation( + ExtractIntervalEpoch("1 second 1 millisecond 1 microsecond"), + Decimal(1.001001, 18, 6)) + } +} 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 index 0d594eb10962e..23ba9c6ec7388 100644 --- 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 @@ -20,9 +20,9 @@ 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.catalyst.util.IntervalUtils 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 { @@ -56,7 +56,7 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { testBothCodegenAndInterpreted("variable-length types") { val proj = createMutableProjection(variableLengthTypes) - val scalaValues = Seq("abc", BigDecimal(10), CalendarInterval.fromString("interval 1 day"), + val scalaValues = Seq("abc", BigDecimal(10), IntervalUtils.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 { 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 a171885471a36..4ccd4f7ce798d 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 @@ -35,8 +35,7 @@ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -486,7 +485,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { ("abcd".getBytes, BinaryType), ("abcd", StringType), (BigDecimal.valueOf(10), DecimalType.IntDecimal), - (CalendarInterval.fromString("interval 3 day"), CalendarIntervalType), + (IntervalUtils.fromString("interval 3 day"), CalendarIntervalType), (java.math.BigDecimal.valueOf(10), DecimalType.BigIntDecimal), (Array(3, 2, 1), ArrayType(IntegerType)) ).foreach { case (input, dt) => 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 b05e49ea33015..52cdd988caa2e 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 @@ -38,7 +38,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { private def booleanLogicTest( name: String, op: (Expression, Expression) => Expression, - truthTable: Seq[(Any, Any, Any)]) { + truthTable: Seq[(Any, Any, Any)]): Unit = { test(s"3VL $name") { truthTable.foreach { case (l, r, answer) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala index 6a3cc21804991..3c826e812b5cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala @@ -508,7 +508,7 @@ class SelectedFieldSuite extends AnalysisTest { // Test that the given SELECT expressions prune the test schema to the single-column schema // defined by the given field private def testSelect(inputSchema: StructType, selectExprs: String*) - (expected: StructField) { + (expected: StructField): Unit = { test(s"SELECT ${selectExprs.map(s => s""""$s"""").mkString(", ")} should select the schema\n" + indent(StructType(expected :: Nil).treeString)) { for (selectExpr <- selectExprs) { 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 d202c2f271d97..e9d2178c0ef22 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 @@ -90,7 +90,7 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with Priva } } - private val parseExpression = PrivateMethod[Long]('parseExpression) + private val parseExpression = PrivateMethod[Long](Symbol("parseExpression")) test("parse sql expression for duration in microseconds - string") { val dur = TimeWindow.invokePrivate(parseExpression(Literal("5 seconds"))) 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 69523fa81bc65..20e77254ecdad 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 @@ -207,7 +207,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB rowWithNoNullColumns.getDecimal(11, 38, 18)) for (i <- fieldTypes.indices) { - // Cann't call setNullAt() on DecimalType + // Can't call setNullAt() on DecimalType if (i == 11) { setToNullAfterCreation.setDecimal(11, null, 38) } else { @@ -531,7 +531,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // 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) + Decimal(255), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2) )) val fields1 = Array( BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, 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 0e0c8e167a0a7..258d0aed94fcb 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 @@ -81,7 +81,7 @@ class PercentileSuite extends SparkFunSuite { private def runTest(agg: Percentile, rows : Seq[Seq[Any]], - expectedPercentiles : Seq[Double]) { + expectedPercentiles : Seq[Double]): Unit = { assert(agg.nullable) val group1 = (0 until rows.length / 2) val group1Buffer = agg.createAggregationBuffer() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerSuite.scala index 75c6beeb32150..81e29931e41dd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerSuite.scala @@ -105,7 +105,8 @@ class GenerateUnsafeRowJoinerSuite extends SparkFunSuite { } } - private def testConcatOnce(numFields1: Int, numFields2: Int, candidateTypes: Seq[DataType]) { + private def testConcatOnce(numFields1: Int, numFields2: Int, + candidateTypes: Seq[DataType]): Unit = { info(s"schema size $numFields1, $numFields2") val random = new Random() val schema1 = RandomDataGenerator.randomSchema(random, numFields1, candidateTypes) @@ -129,7 +130,7 @@ class GenerateUnsafeRowJoinerSuite extends SparkFunSuite { schema1: StructType, row1: UnsafeRow, schema2: StructType, - row2: UnsafeRow) { + row2: UnsafeRow): Unit = { // Run the joiner. val mergedSchema = StructType(schema1 ++ schema2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala new file mode 100644 index 0000000000000..175904da21969 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.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.catalyst.expressions.postgreSQL + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} + +class CastSuite extends SparkFunSuite with ExpressionEvalHelper { + private def checkPostgreCastStringToBoolean(v: Any, expected: Any): Unit = { + checkEvaluation(PostgreCastStringToBoolean(Literal(v)), expected) + } + + test("cast string to boolean") { + checkPostgreCastStringToBoolean("true", true) + checkPostgreCastStringToBoolean("tru", true) + checkPostgreCastStringToBoolean("tr", true) + checkPostgreCastStringToBoolean("t", true) + checkPostgreCastStringToBoolean("tRUe", true) + checkPostgreCastStringToBoolean(" tRue ", true) + checkPostgreCastStringToBoolean(" tRu ", true) + checkPostgreCastStringToBoolean("yes", true) + checkPostgreCastStringToBoolean("ye", true) + checkPostgreCastStringToBoolean("y", true) + checkPostgreCastStringToBoolean("1", true) + checkPostgreCastStringToBoolean("on", true) + + checkPostgreCastStringToBoolean("false", false) + checkPostgreCastStringToBoolean("fals", false) + checkPostgreCastStringToBoolean("fal", false) + checkPostgreCastStringToBoolean("fa", false) + checkPostgreCastStringToBoolean("f", false) + checkPostgreCastStringToBoolean(" fAlse ", false) + checkPostgreCastStringToBoolean(" fAls ", false) + checkPostgreCastStringToBoolean(" FAlsE ", false) + checkPostgreCastStringToBoolean("no", false) + checkPostgreCastStringToBoolean("n", false) + checkPostgreCastStringToBoolean("0", false) + checkPostgreCastStringToBoolean("off", false) + checkPostgreCastStringToBoolean("of", false) + + checkPostgreCastStringToBoolean("o", null) + checkPostgreCastStringToBoolean("abc", null) + checkPostgreCastStringToBoolean("", null) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala index 9b27490ed0e35..2bb948ec24fb3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala @@ -39,6 +39,33 @@ class JacksonGeneratorSuite extends SparkFunSuite { assert(writer.toString === """{"a":1}""") } + test("SPARK-29444: initial with StructType and write out an empty row " + + "with ignoreNullFields=false") { + val dataType = StructType(StructField("a", IntegerType) :: Nil) + val input = InternalRow(null) + val writer = new CharArrayWriter() + val allowNullOption = + new JSONOptions(Map("ignoreNullFields" -> "false"), gmtId) + val gen = new JacksonGenerator(dataType, writer, allowNullOption) + gen.write(input) + gen.flush() + assert(writer.toString === """{"a":null}""") + } + + test("SPARK-29444: initial with StructType field and write out a row " + + "with ignoreNullFields=false and struct inner null") { + val fieldType = StructType(StructField("b", IntegerType) :: Nil) + val dataType = StructType(StructField("a", fieldType) :: Nil) + val input = InternalRow(InternalRow(null)) + val writer = new CharArrayWriter() + val allowNullOption = + new JSONOptions(Map("ignoreNullFields" -> "false"), gmtId) + val gen = new JacksonGenerator(dataType, writer, allowNullOption) + gen.write(input) + gen.flush() + assert(writer.toString === """{"a":{"b":null}}""") + } + test("initial with StructType and write out rows") { val dataType = StructType(StructField("a", IntegerType) :: Nil) val input = new GenericArrayData(InternalRow(1) :: InternalRow(2) :: Nil) 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 75ff07637fccc..5be37318ae6eb 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 @@ -109,7 +109,7 @@ class ColumnPruningSuite extends PlanTest { replacedGenerator: Seq[String] => Generator, aliasedExprs: Seq[String] => Seq[Expression], unrequiredChildIndex: Seq[Int], - generatorOutputNames: Seq[String]) { + generatorOutputNames: Seq[String]): Unit = { withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { val structType = StructType.fromDDL("d double, e array, f double, g double, " + "h array>") 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 0f93305565224..3d81c567eff11 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 @@ -52,7 +52,8 @@ class JoinOptimizationSuite extends PlanTest { val y = testRelation1.subquery('y) val z = testRelation.subquery('z) - def testExtract(plan: LogicalPlan, expected: Option[(Seq[LogicalPlan], Seq[Expression])]) { + def testExtract(plan: LogicalPlan, + expected: Option[(Seq[LogicalPlan], Seq[Expression])]): Unit = { val expectedNoCross = expected map { seq_pair => { val plans = seq_pair._1 @@ -63,8 +64,8 @@ class JoinOptimizationSuite extends PlanTest { testExtractCheckCross(plan, expectedNoCross) } - def testExtractCheckCross - (plan: LogicalPlan, expected: Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])]) { + def testExtractCheckCross(plan: LogicalPlan, + expected: Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])]): Unit = { assert( ExtractFiltersAndInnerJoins.unapply(plan) === expected.map(e => (e._1, e._2))) } 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 2a8780346d99e..a277a2d339e91 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 @@ -28,7 +28,7 @@ class OptimizerRuleExclusionSuite extends PlanTest { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - private def verifyExcludedRules(optimizer: Optimizer, rulesToExclude: Seq[String]) { + private def verifyExcludedRules(optimizer: Optimizer, rulesToExclude: Seq[String]): Unit = { val nonExcludableRules = optimizer.nonExcludableRules val excludedRuleNames = rulesToExclude.filter(!nonExcludableRules.contains(_)) 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 5e0d2041fac5d..5998437f11f4d 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, FakeV2SessionCatalog, 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._ @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, NamedExpressio import org.apache.spark.sql.catalyst.plans.PlanTest 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.connector.catalog.CatalogManager import org.apache.spark.sql.internal.SQLConf @@ -43,10 +44,10 @@ class OptimizerStructuralIntegrityCheckerSuite extends PlanTest { } object Optimize extends Optimizer( - new SessionCatalog( - new InMemoryCatalog, - EmptyFunctionRegistry, - new SQLConf())) { + new CatalogManager( + new SQLConf(), + FakeV2SessionCatalog, + new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, new SQLConf()))) { val newBatch = Batch("OptimizeRuleBreakSI", Once, OptimizeRuleBreakSI) override def defaultBatches: Seq[Batch] = Seq(newBatch) ++ super.defaultBatches } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index be781081b0112..f4375956f0af6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -23,8 +23,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -408,7 +407,7 @@ class DDLParserSuite extends AnalysisTest { private def testCreateOrReplaceDdl( sqlStatement: String, tableSpec: TableSpec, - expectedIfNotExists: Boolean) { + expectedIfNotExists: Boolean): Unit = { val parsedPlan = parsePlan(sqlStatement) val newTableToken = sqlStatement.split(" ")(0).trim.toUpperCase(Locale.ROOT) parsedPlan match { @@ -789,6 +788,48 @@ class DDLParserSuite extends AnalysisTest { assert(exc.getMessage.contains("Columns aliases is not allowed in DELETE.")) } + test("update table: basic") { + parseCompare( + """ + |UPDATE testcat.ns1.ns2.tbl + |SET t.a='Robert', t.b=32 + """.stripMargin, + UpdateTableStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + None, + Seq(Seq("t", "a"), Seq("t", "b")), + Seq(Literal("Robert"), Literal(32)), + None)) + } + + test("update table: with alias and where clause") { + parseCompare( + """ + |UPDATE testcat.ns1.ns2.tbl AS t + |SET t.a='Robert', t.b=32 + |WHERE t.c=2 + """.stripMargin, + UpdateTableStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + Some("t"), + Seq(Seq("t", "a"), Seq("t", "b")), + Seq(Literal("Robert"), Literal(32)), + Some(EqualTo(UnresolvedAttribute("t.c"), Literal(2))))) + } + + test("update table: columns aliases is not allowed") { + val exc = intercept[ParseException] { + parsePlan( + """ + |UPDATE testcat.ns1.ns2.tbl AS t(a,b,c,d) + |SET b='Robert', c=32 + |WHERE d=2 + """.stripMargin) + } + + assert(exc.getMessage.contains("Columns aliases is not allowed in UPDATE.")) + } + test("show tables") { comparePlans( parsePlan("SHOW TABLES"), @@ -804,6 +845,109 @@ class DDLParserSuite extends AnalysisTest { ShowTablesStatement(Some(Seq("tbl")), Some("*dog*"))) } + test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { + val expected = CreateNamespaceStatement( + Seq("a", "b", "c"), + ifNotExists = true, + Map( + "a" -> "a", + "b" -> "b", + "c" -> "c", + "comment" -> "namespace_comment", + "location" -> "/home/user/db")) + + comparePlans( + parsePlan( + """ + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'namespace_comment' LOCATION '/home/user/db' + """.stripMargin), + expected) + + comparePlans( + parsePlan( + """ + |CREATE DATABASE IF NOT EXISTS a.b.c + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'namespace_comment' LOCATION '/home/user/db' + """.stripMargin), + expected) + } + + test("create namespace -- check duplicates") { + def createDatabase(duplicateClause: String): String = { + s""" + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |$duplicateClause + |$duplicateClause + """.stripMargin + } + val sql1 = createDatabase("COMMENT 'namespace_comment'") + val sql2 = createDatabase("LOCATION '/home/user/db'") + val sql3 = createDatabase("WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c')") + val sql4 = 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 PROPERTIES") + intercept(sql4, "Found duplicate clauses: WITH DBPROPERTIES") + } + + test("create namespace - property values must be set") { + assertUnsupported( + sql = "CREATE NAMESPACE a.b.c WITH PROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("create namespace -- either PROPERTIES or DBPROPERTIES is allowed") { + val sql = + s""" + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + """.stripMargin + intercept(sql, "Either PROPERTIES or DBPROPERTIES is allowed") + } + + test("create namespace - support for other types in PROPERTIES") { + val sql = + """ + |CREATE NAMESPACE a.b.c + |LOCATION '/home/user/db' + |WITH PROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) + """.stripMargin + comparePlans( + parsePlan(sql), + CreateNamespaceStatement( + Seq("a", "b", "c"), + ifNotExists = false, + Map( + "a" -> "1", + "b" -> "0.1", + "c" -> "true", + "location" -> "/home/user/db"))) + } + + test("show databases: basic") { + comparePlans( + parsePlan("SHOW DATABASES"), + ShowNamespacesStatement(None, None)) + comparePlans( + parsePlan("SHOW DATABASES LIKE 'defau*'"), + ShowNamespacesStatement(None, Some("defau*"))) + } + + test("show databases: FROM/IN operator is not allowed") { + def verify(sql: String): Unit = { + val exc = intercept[ParseException] { parsePlan(sql) } + assert(exc.getMessage.contains("FROM/IN operator is not allowed in SHOW DATABASES")) + } + + verify("SHOW DATABASES FROM testcat.ns1.ns2") + verify("SHOW DATABASES IN testcat.ns1.ns2") + } + test("show namespaces") { comparePlans( parsePlan("SHOW NAMESPACES"), @@ -819,6 +963,157 @@ class DDLParserSuite extends AnalysisTest { ShowNamespacesStatement(Some(Seq("testcat", "ns1")), Some("*pattern*"))) } + test("analyze table statistics") { + comparePlans(parsePlan("analyze table a.b.c compute statistics"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map.empty, noScan = false)) + comparePlans(parsePlan("analyze table a.b.c compute statistics noscan"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map.empty, noScan = true)) + comparePlans(parsePlan("analyze table a.b.c partition (a) compute statistics nOscAn"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("a" -> None), noScan = true)) + + // Partitions specified + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = false)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09")), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = false)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr=11) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> None, "hr" -> Some("11")), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("ds" -> None, "hr" -> None), noScan = false)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("ds" -> None, "hr" -> None), noScan = true)) + + intercept("analyze table a.b.c compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + intercept("analyze table a.b.c partition (a) compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + } + + test("analyze table column statistics") { + intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS", "") + + comparePlans( + parsePlan("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS key, value"), + AnalyzeColumnStatement(Seq("a", "b", "c"), Option(Seq("key", "value")), allColumns = false)) + + // Partition specified - should be ignored + comparePlans( + parsePlan( + s""" + |ANALYZE TABLE a.b.c PARTITION(ds='2017-06-10') + |COMPUTE STATISTICS FOR COLUMNS key, value + """.stripMargin), + AnalyzeColumnStatement(Seq("a", "b", "c"), Option(Seq("key", "value")), allColumns = false)) + + // Partition specified should be ignored in case of COMPUTE STATISTICS FOR ALL COLUMNS + comparePlans( + parsePlan( + s""" + |ANALYZE TABLE a.b.c PARTITION(ds='2017-06-10') + |COMPUTE STATISTICS FOR ALL COLUMNS + """.stripMargin), + AnalyzeColumnStatement(Seq("a", "b", "c"), None, allColumns = true)) + + intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value", + "mismatched input 'key' expecting ") + intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL", + "missing 'COLUMNS' at ''") + } + + test("MSCK REPAIR TABLE") { + comparePlans( + parsePlan("MSCK REPAIR TABLE a.b.c"), + RepairTableStatement(Seq("a", "b", "c"))) + } + + test("CACHE TABLE") { + comparePlans( + parsePlan("CACHE TABLE a.b.c"), + CacheTableStatement(Seq("a", "b", "c"), None, false, Map.empty)) + + comparePlans( + parsePlan("CACHE LAZY TABLE a.b.c"), + CacheTableStatement(Seq("a", "b", "c"), None, true, Map.empty)) + + comparePlans( + parsePlan("CACHE LAZY TABLE a.b.c OPTIONS('storageLevel' 'DISK_ONLY')"), + CacheTableStatement(Seq("a", "b", "c"), None, true, Map("storageLevel" -> "DISK_ONLY"))) + + intercept("CACHE TABLE a.b.c AS SELECT * FROM testData", + "It is not allowed to add catalog/namespace prefix a.b") + } + + test("UNCACHE TABLE") { + comparePlans( + parsePlan("UNCACHE TABLE a.b.c"), + UncacheTableStatement(Seq("a", "b", "c"), ifExists = false)) + + comparePlans( + parsePlan("UNCACHE TABLE IF EXISTS a.b.c"), + UncacheTableStatement(Seq("a", "b", "c"), ifExists = true)) + } + + test("TRUNCATE table") { + comparePlans( + parsePlan("TRUNCATE TABLE a.b.c"), + TruncateTableStatement(Seq("a", "b", "c"), None)) + + comparePlans( + parsePlan("TRUNCATE TABLE a.b.c PARTITION(ds='2017-06-10')"), + TruncateTableStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10")))) + } + + test("SHOW PARTITIONS") { + val sql1 = "SHOW PARTITIONS t1" + val sql2 = "SHOW PARTITIONS db1.t1" + val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" + val sql4 = "SHOW PARTITIONS a.b.c" + val sql5 = "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" + + val parsed1 = parsePlan(sql1) + val expected1 = ShowPartitionsStatement(Seq("t1"), None) + val parsed2 = parsePlan(sql2) + val expected2 = ShowPartitionsStatement(Seq("db1", "t1"), None) + val parsed3 = parsePlan(sql3) + val expected3 = ShowPartitionsStatement(Seq("t1"), + Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))) + val parsed4 = parsePlan(sql4) + val expected4 = ShowPartitionsStatement(Seq("a", "b", "c"), None) + val parsed5 = parsePlan(sql5) + val expected5 = ShowPartitionsStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10"))) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("REFRESH TABLE") { + comparePlans( + parsePlan("REFRESH TABLE a.b.c"), + RefreshTableStatement(Seq("a", "b", "c"))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], 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 5da2bf059758d..86b3aa8190b45 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 @@ -24,7 +24,7 @@ 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.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -424,18 +424,18 @@ class ExpressionParserSuite extends AnalysisTest { test("type constructors") { // Dates. assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11"))) - intercept("DAtE 'mar 11 2016'") + intercept("DAtE 'mar 11 2016'", "Cannot parse the DATE value") // Timestamps. assertEqual("tImEstAmp '2016-03-11 20:54:00.000'", Literal(Timestamp.valueOf("2016-03-11 20:54:00.000"))) - intercept("timestamP '2016-33-11 20:54:00.000'") + intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value") // Interval. - assertEqual("InterVal 'interval 3 month 1 hour'", - Literal(CalendarInterval.fromString("interval 3 month 1 hour"))) - assertEqual("Interval 'interval 3 monthsss 1 hoursss'", - Literal(null, CalendarIntervalType)) + val intervalLiteral = Literal(IntervalUtils.fromString("interval 3 month 1 hour")) + assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral) + assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) + intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") // Binary. assertEqual("X'A'", Literal(Array(0x0a).map(_.toByte))) @@ -597,7 +597,7 @@ class ExpressionParserSuite extends AnalysisTest { "microsecond") def intervalLiteral(u: String, s: String): Literal = { - Literal(CalendarInterval.fromSingleUnitString(u, s)) + Literal(CalendarInterval.fromUnitStrings(Array(u), Array(s))) } test("intervals") { 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 cba24fbe2a65b..c6434f2bdd3ec 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 @@ -40,7 +40,7 @@ class ParserUtilsSuite extends SparkFunSuite { } val showDbsContext = buildContext("show databases like 'identifier_with_wildcards'") { parser => - parser.statement().asInstanceOf[ShowDatabasesContext] + parser.statement().asInstanceOf[ShowNamespacesContext] } val createDbContext = buildContext( @@ -50,7 +50,7 @@ class ParserUtilsSuite extends SparkFunSuite { |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') """.stripMargin ) { parser => - parser.statement().asInstanceOf[CreateDatabaseContext] + parser.statement().asInstanceOf[CreateNamespaceContext] } val emptyContext = buildContext("") { parser => 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 61f8c3b99149a..8b940a7aa2c3b 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 @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, Un 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.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType 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 08f1f87514b1d..b28e6ded6ca98 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 @@ -160,7 +160,7 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => } /** Fails the test if the join order in the two plans do not match */ - protected def compareJoinOrder(plan1: LogicalPlan, plan2: LogicalPlan) { + protected def compareJoinOrder(plan1: LogicalPlan, plan2: LogicalPlan): Unit = { val normalized1 = normalizePlan(normalizeExprIds(plan1)) val normalized2 = normalizePlan(normalizeExprIds(plan2)) if (!sameJoinPlan(normalized1, normalized2)) { 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 828706743c5b5..0e094bc06b05f 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 @@ -28,7 +28,7 @@ import org.json4s.jackson.JsonMethods import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.{AliasIdentifier, FunctionIdentifier, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions.DslString import org.apache.spark.sql.catalyst.expressions._ @@ -431,6 +431,28 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { "product-class" -> JString(classOf[FunctionIdentifier].getName), "funcName" -> "function")) + // Converts AliasIdentifier to JSON + assertJSON( + AliasIdentifier("alias"), + JObject( + "product-class" -> JString(classOf[AliasIdentifier].getName), + "identifier" -> "alias")) + + // Converts SubqueryAlias to JSON + assertJSON( + SubqueryAlias("t1", JsonTestTreeNode("0")), + List( + JObject( + "class" -> classOf[SubqueryAlias].getName, + "num-children" -> 1, + "name" -> JObject("product-class" -> JString(classOf[AliasIdentifier].getName), + "identifier" -> "t1"), + "child" -> 0), + JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "0"))) + // Converts BucketSpec to JSON assertJSON( BucketSpec(1, Seq("bucket"), Seq("sort")), 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 1da8efe4ef42c..10642b3ca8a4f 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 @@ -37,7 +37,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { test("nanoseconds truncation") { val tf = TimestampFormatter.getFractionFormatter(DateTimeUtils.defaultTimeZone.toZoneId) - def checkStringToTimestamp(originalTime: String, expectedParsedTime: String) { + def checkStringToTimestamp(originalTime: String, expectedParsedTime: String): Unit = { val parsedTimestampOp = DateTimeUtils.stringToTimestamp( UTF8String.fromString(originalTime), defaultZoneId) assert(parsedTimestampOp.isDefined, "timestamp with nanoseconds was not parsed correctly") @@ -456,6 +456,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { } } + test("trailing characters while converting string to timestamp") { + val s = UTF8String.fromString("2019-10-31T10:59:23Z:::") + val time = DateTimeUtils.stringToTimestamp(s, defaultZoneId) + assert(time == None) + } + test("truncTimestamp") { def testTrunc( level: Int, @@ -549,12 +555,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( - "Kwajalein" -> Set(8632, 8633), + "Kwajalein" -> Set(8632, 8633, 8634), "Pacific/Apia" -> Set(15338), "Pacific/Enderbury" -> Set(9130, 9131), "Pacific/Fakaofo" -> Set(15338), "Pacific/Kiritimati" -> Set(9130, 9131), - "Pacific/Kwajalein" -> Set(8632, 8633), + "Pacific/Kwajalein" -> Set(8632, 8633, 8634), "MIT" -> Set(15338)) for (tz <- ALL_TIMEZONES) { val skipped = skipped_days.getOrElse(tz.getID, Set.empty) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala new file mode 100644 index 0000000000000..e48779af3c9aa --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.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.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.IntervalUtils.fromString +import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.CalendarInterval._ + +class IntervalUtilsSuite extends SparkFunSuite { + + test("fromString: basic") { + testSingleUnit("YEAR", 3, 36, 0) + testSingleUnit("Month", 3, 3, 0) + testSingleUnit("Week", 3, 0, 3 * MICROS_PER_WEEK) + testSingleUnit("DAY", 3, 0, 3 * MICROS_PER_DAY) + testSingleUnit("HouR", 3, 0, 3 * MICROS_PER_HOUR) + testSingleUnit("MiNuTe", 3, 0, 3 * MICROS_PER_MINUTE) + testSingleUnit("Second", 3, 0, 3 * MICROS_PER_SECOND) + testSingleUnit("MilliSecond", 3, 0, 3 * MICROS_PER_MILLI) + testSingleUnit("MicroSecond", 3, 0, 3) + + for (input <- Seq(null, "", " ")) { + try { + fromString(input) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + val msg = e.getMessage + if (input == null) { + assert(msg.contains("cannot be null")) + } + } + } + + for (input <- Seq("interval", "interval1 day", "foo", "foo 1 day")) { + try { + fromString(input) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + val msg = e.getMessage + assert(msg.contains("Invalid interval string")) + } + } + } + + test("fromString: random order field") { + val input = "1 day 1 year" + val result = new CalendarInterval(12, MICROS_PER_DAY) + assert(fromString(input) == result) + } + + test("fromString: duplicated fields") { + val input = "1 day 1 day" + val result = new CalendarInterval(0, 2 * MICROS_PER_DAY) + assert(fromString(input) == result) + } + + test("fromString: value with +/-") { + val input = "+1 year -1 day" + val result = new CalendarInterval(12, -MICROS_PER_DAY) + assert(fromString(input) == result) + } + + private def testSingleUnit(unit: String, number: Int, months: Int, microseconds: Long): Unit = { + for (prefix <- Seq("interval ", "")) { + val input1 = prefix + number + " " + unit + val input2 = prefix + number + " " + unit + "s" + val result = new CalendarInterval(months, microseconds) + assert(fromString(input1) == result) + assert(fromString(input2) == result) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala index 650813975d75c..e53d0bbccc614 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala @@ -169,5 +169,22 @@ class QuantileSummariesSuite extends SparkFunSuite { checkQuantile(0.1, data, s) checkQuantile(0.001, data, s) } + + // length of data21 is 4 * length of data22 + val data21 = data.zipWithIndex.filter(_._2 % 5 != 0).map(_._1).toSeq + val data22 = data.zipWithIndex.filter(_._2 % 5 == 0).map(_._1).toSeq + + test( + s"Merging unbalanced interleaved lists with epsi=$epsi and seq=$seq_name, " + + s"compression=$compression") { + val s1 = buildSummary(data21, epsi, compression) + val s2 = buildSummary(data22, epsi, compression) + val s = s1.merge(s2) + // Check all quantiles + for (queryRank <- 1 to n) { + val queryQuantile = queryRank.toDouble / n.toDouble + checkQuantile(queryQuantile, data, s) + } + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index 8724a38d08d1f..ece903a4c2838 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -34,6 +34,8 @@ class BasicInMemoryTableCatalog extends TableCatalog { protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() + private val invalidatedTables: util.Set[Identifier] = ConcurrentHashMap.newKeySet() + private var _name: Option[String] = None override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { @@ -55,6 +57,10 @@ class BasicInMemoryTableCatalog extends TableCatalog { } } + override def invalidateTable(ident: Identifier): Unit = { + invalidatedTables.add(ident) + } + override def createTable( ident: Identifier, schema: StructType, @@ -104,6 +110,10 @@ class BasicInMemoryTableCatalog extends TableCatalog { } } + def isTableInvalidated(ident: Identifier): Boolean = { + invalidatedTables.contains(ident) + } + def clearTables(): Unit = { tables.clear() } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index 58e2f410bf205..fcbf139fd97c1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -17,53 +17,98 @@ package org.apache.spark.sql.connector.catalog +import java.net.URI import java.util import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog +import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, FakeV2SessionCatalog, NoSuchNamespaceException} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap class CatalogManagerSuite extends SparkFunSuite { + private def createSessionCatalog(conf: SQLConf): SessionCatalog = { + val catalog = new InMemoryCatalog() + catalog.createDatabase( + CatalogDatabase(SessionCatalog.DEFAULT_DATABASE, "", new URI("fake"), Map.empty), + ignoreIfExists = true) + new SessionCatalog(catalog, EmptyFunctionRegistry, conf) + } + test("CatalogManager should reflect the changes of default catalog") { val conf = new SQLConf - val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog) - assert(catalogManager.currentCatalog.isEmpty) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) + assert(catalogManager.currentCatalog.name() == CatalogManager.SESSION_CATALOG_NAME) assert(catalogManager.currentNamespace.sameElements(Array("default"))) conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) conf.setConfString(SQLConf.DEFAULT_V2_CATALOG.key, "dummy") // The current catalog should be changed if the default catalog is set. - assert(catalogManager.currentCatalog == Some("dummy")) + assert(catalogManager.currentCatalog.name() == "dummy") assert(catalogManager.currentNamespace.sameElements(Array("a", "b"))) } test("CatalogManager should keep the current catalog once set") { val conf = new SQLConf - val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog) - assert(catalogManager.currentCatalog.isEmpty) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) + assert(catalogManager.currentCatalog.name() == CatalogManager.SESSION_CATALOG_NAME) conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) catalogManager.setCurrentCatalog("dummy") - assert(catalogManager.currentCatalog == Some("dummy")) + assert(catalogManager.currentCatalog.name() == "dummy") assert(catalogManager.currentNamespace.sameElements(Array("a", "b"))) conf.setConfString("spark.sql.catalog.dummy2", classOf[DummyCatalog].getName) conf.setConfString(SQLConf.DEFAULT_V2_CATALOG.key, "dummy2") // The current catalog shouldn't be changed if it's set before. - assert(catalogManager.currentCatalog == Some("dummy")) + assert(catalogManager.currentCatalog.name() == "dummy") } test("current namespace should be updated when switching current catalog") { val conf = new SQLConf - val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog) - catalogManager.setCurrentNamespace(Array("abc")) - assert(catalogManager.currentNamespace.sameElements(Array("abc"))) - + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) catalogManager.setCurrentCatalog("dummy") assert(catalogManager.currentNamespace.sameElements(Array("a", "b"))) + catalogManager.setCurrentNamespace(Array("a")) + assert(catalogManager.currentNamespace.sameElements(Array("a"))) + + // If we set current catalog to the same catalog, current namespace should stay the same. + catalogManager.setCurrentCatalog("dummy") + assert(catalogManager.currentNamespace.sameElements(Array("a"))) + + // If we switch to a different catalog, current namespace should be reset. + conf.setConfString("spark.sql.catalog.dummy2", classOf[DummyCatalog].getName) + catalogManager.setCurrentCatalog("dummy2") + assert(catalogManager.currentNamespace.sameElements(Array("a", "b"))) + } + + test("set current namespace") { + val conf = new SQLConf + val v1SessionCatalog = createSessionCatalog(conf) + v1SessionCatalog.createDatabase( + CatalogDatabase( + "test", "", v1SessionCatalog.getDefaultDBPath("test"), Map.empty), + ignoreIfExists = false) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, v1SessionCatalog) + + // If the current catalog is session catalog, setting current namespace actually sets + // `SessionCatalog.currentDb`. + catalogManager.setCurrentNamespace(Array("test")) + assert(catalogManager.currentNamespace.sameElements(Array("test"))) + assert(v1SessionCatalog.getCurrentDatabase == "test") + + intercept[NoSuchNamespaceException] { + catalogManager.setCurrentNamespace(Array("ns1", "ns2")) + } + + // when switching current catalog, `SessionCatalog.currentDb` should be reset. + conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) + catalogManager.setCurrentCatalog("dummy") + assert(v1SessionCatalog.getCurrentDatabase == "default") + catalogManager.setCurrentNamespace(Array("test2")) + assert(v1SessionCatalog.getCurrentDatabase == "default") } } @@ -89,6 +134,9 @@ class DummyCatalog extends SupportsNamespaces { override def dropNamespace(namespace: Array[String]): Boolean = { throw new UnsupportedOperationException } - override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {} - override def name(): String = "dummy" + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + _name = name + } + private var _name: String = null + override def name(): String = _name } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index 187551dd90db7..513f7e0348d09 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -24,18 +24,20 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.util.CaseInsensitiveStringMap private case class DummyCatalogPlugin(override val name: String) extends CatalogPlugin { - override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = Unit + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = () } class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap + private val sessionCatalog = FakeV2SessionCatalog override val catalogManager: CatalogManager = { val manager = mock(classOf[CatalogManager]) @@ -43,22 +45,22 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { val name = invocation.getArgument[String](0) catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) }) - when(manager.defaultCatalog).thenReturn(None) + when(manager.currentCatalog).thenReturn(sessionCatalog) manager } 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, + ("tbl", sessionCatalog, Seq.empty, "tbl"), + ("db.tbl", sessionCatalog, Seq("db"), "tbl"), + ("prod.func", catalogs("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", sessionCatalog, Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", sessionCatalog, Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", sessionCatalog, Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", sessionCatalog, Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { case (sql, expectedCatalog, namespace, name) => inside(parseMultipartIdentifier(sql)) { @@ -135,22 +137,22 @@ class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog wit val name = invocation.getArgument[String](0) catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) }) - when(manager.defaultCatalog).thenReturn(catalogs.get("prod")) + when(manager.currentCatalog).thenReturn(catalogs("prod")) manager } 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"), + ("tbl", catalogs("prod"), Seq.empty, "tbl"), + ("db.tbl", catalogs("prod"), Seq("db"), "tbl"), + ("prod.func", catalogs("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", catalogs("prod"), Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", catalogs("prod"), Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", catalogs("prod"), Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", catalogs("prod"), Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { case (sql, expectedCatalog, namespace, name) => inside(parseMultipartIdentifier(sql)) { 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 9d6827194f004..c47332f5d9fcb 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 @@ -76,6 +76,14 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa assert(err.contains("Cannot safely cast")) } } + + test("Check NullType is incompatible with all other types") { + allNonNullTypes.foreach { t => + assertSingleError(NullType, t, "nulls", s"Should not allow writing None to type $t") { err => + assert(err.contains(s"incompatible with $t")) + } + } + } } class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBaseSuite { @@ -145,6 +153,12 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase assert(err.contains("Cannot safely cast 'timestampToLong': TimestampType to LongType")) } } + + test("Check NullType is compatible with all other types") { + allNonNullTypes.foreach { t => + assertAllowed(NullType, t, "nulls", s"Should allow writing None to type $t") + } + } } abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { @@ -175,17 +189,9 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { private val nestedContainerTypes = Seq(ArrayType(point2, containsNull = false), MapType(StringType, point3, valueContainsNull = false)) - private val allNonNullTypes = Seq( + protected val allNonNullTypes = Seq( atomicTypes, simpleContainerTypes, nestedContainerTypes, Seq(CalendarIntervalType)).flatten - test("Check NullType is incompatible with all other types") { - allNonNullTypes.foreach { t => - assertSingleError(NullType, t, "nulls", s"Should not allow writing None to type $t") { err => - assert(err.contains(s"incompatible with $t")) - } - } - } - test("Check each type with itself") { allNonNullTypes.foreach { t => assertAllowed(t, t, "t", s"Should allow writing type to itself $t") 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 d69bb2f0b6bcc..bf41b711c9473 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 @@ -99,7 +99,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { } // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs - private val decimalVal = PrivateMethod[BigDecimal]('decimalVal) + private val decimalVal = PrivateMethod[BigDecimal](Symbol("decimalVal")) /** Check whether a decimal is represented compactly (passing whether we expect it to be) */ private def checkCompact(d: Decimal, expected: Boolean): Unit = { diff --git a/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt b/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..e71d7ab0f3b6f --- /dev/null +++ b/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt @@ -0,0 +1,143 @@ +================================================================================================ +aggregate without grouping +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +agg w/o group wholestage off 55644 59484 NaN 37.7 26.5 1.0X +agg w/o group wholestage on 896 906 8 2340.7 0.4 62.1X + + +================================================================================================ +stat functions +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +stddev wholestage off 8655 9022 519 12.1 82.5 1.0X +stddev wholestage on 1306 1323 13 80.3 12.5 6.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +kurtosis wholestage off 40795 41041 349 2.6 389.0 1.0X +kurtosis wholestage on 1441 1468 22 72.8 13.7 28.3X + + +================================================================================================ +aggregate with linear keys +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 10559 10956 562 7.9 125.9 1.0X +codegen = T hashmap = F 6533 6567 34 12.8 77.9 1.6X +codegen = T hashmap = T 1362 1377 22 61.6 16.2 7.8X + + +================================================================================================ +aggregate with randomized keys +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 12631 12634 3 6.6 150.6 1.0X +codegen = T hashmap = F 8434 8478 44 9.9 100.5 1.5X +codegen = T hashmap = T 2484 2598 117 33.8 29.6 5.1X + + +================================================================================================ +aggregate with string key +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 4173 4204 44 5.0 199.0 1.0X +codegen = T hashmap = F 2664 2721 81 7.9 127.0 1.6X +codegen = T hashmap = T 1178 1219 59 17.8 56.2 3.5X + + +================================================================================================ +aggregate with decimal key +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 3740 3747 9 5.6 178.3 1.0X +codegen = T hashmap = F 2398 2528 184 8.7 114.3 1.6X +codegen = T hashmap = T 638 644 7 32.9 30.4 5.9X + + +================================================================================================ +aggregate with multiple key types +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 6874 6896 31 3.1 327.8 1.0X +codegen = T hashmap = F 3866 3886 28 5.4 184.3 1.8X +codegen = T hashmap = T 2619 2641 31 8.0 124.9 2.6X + + +================================================================================================ +max function bytecode size of wholestagecodegen +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 793 806 19 0.8 1209.7 1.0X +codegen = T hugeMethodLimit = 10000 401 456 61 1.6 611.2 2.0X +codegen = T hugeMethodLimit = 1500 694 715 19 0.9 1059.3 1.1X + + +================================================================================================ +cube +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cube wholestage off 3616 3630 19 1.4 689.8 1.0X +cube wholestage on 1819 1866 41 2.9 347.0 2.0X + + +================================================================================================ +hash and BytesToBytesMap +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UnsafeRowhash 321 321 1 65.4 15.3 1.0X +murmur3 hash 145 145 0 144.5 6.9 2.2X +fast hash 70 71 1 298.9 3.3 4.6X +arrayEqual 198 199 2 105.7 9.5 1.6X +Java HashMap (Long) 132 136 3 158.6 6.3 2.4X +Java HashMap (two ints) 152 156 4 138.3 7.2 2.1X +Java HashMap (UnsafeRow) 819 839 17 25.6 39.1 0.4X +LongToUnsafeRowMap (opt=false) 459 472 25 45.7 21.9 0.7X +LongToUnsafeRowMap (opt=true) 107 108 1 195.5 5.1 3.0X +BytesToBytesMap (off Heap) 1012 1019 10 20.7 48.3 0.3X +BytesToBytesMap (on Heap) 963 974 18 21.8 45.9 0.3X +Aggregate HashMap 41 43 2 515.1 1.9 7.9X + + diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index 19e524777692e..ea8f113143a0f 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -2,142 +2,142 @@ aggregate without grouping ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -agg w/o group wholestage off 65374 / 70665 32.1 31.2 1.0X -agg w/o group wholestage on 1178 / 1209 1779.8 0.6 55.5X +agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +agg w/o group wholestage off 50499 52524 2863 41.5 24.1 1.0X +agg w/o group wholestage on 1163 1205 56 1803.1 0.6 43.4X ================================================================================================ stat functions ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -stddev wholestage off 8667 / 8851 12.1 82.7 1.0X -stddev wholestage on 1266 / 1273 82.8 12.1 6.8X +stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +stddev wholestage off 13393 13403 14 7.8 127.7 1.0X +stddev wholestage on 1291 1315 25 81.2 12.3 10.4X -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -kurtosis wholestage off 41218 / 41231 2.5 393.1 1.0X -kurtosis wholestage on 1347 / 1357 77.8 12.8 30.6X +kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +kurtosis wholestage off 40131 40250 168 2.6 382.7 1.0X +kurtosis wholestage on 1435 1452 20 73.1 13.7 28.0X ================================================================================================ aggregate with linear keys ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -codegen = F 9309 / 9389 9.0 111.0 1.0X -codegen = T hashmap = F 4417 / 4435 19.0 52.7 2.1X -codegen = T hashmap = T 1289 / 1298 65.1 15.4 7.2X +Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 10008 10008 1 8.4 119.3 1.0X +codegen = T hashmap = F 5803 6158 370 14.5 69.2 1.7X +codegen = T hashmap = T 1332 1348 12 63.0 15.9 7.5X ================================================================================================ aggregate with randomized keys ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -codegen = F 11424 / 11426 7.3 136.2 1.0X -codegen = T hashmap = F 6441 / 6496 13.0 76.8 1.8X -codegen = T hashmap = T 2333 / 2344 36.0 27.8 4.9X +Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 11853 11988 191 7.1 141.3 1.0X +codegen = T hashmap = F 7469 7531 57 11.2 89.0 1.6X +codegen = T hashmap = T 2412 2436 24 34.8 28.8 4.9X ================================================================================================ aggregate with string key ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -codegen = F 4751 / 4890 4.4 226.5 1.0X -codegen = T hashmap = F 3146 / 3182 6.7 150.0 1.5X -codegen = T hashmap = T 2211 / 2261 9.5 105.4 2.1X +Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 4880 4908 40 4.3 232.7 1.0X +codegen = T hashmap = F 3483 3505 26 6.0 166.1 1.4X +codegen = T hashmap = T 2362 2425 39 8.9 112.6 2.1X ================================================================================================ aggregate with decimal key ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -codegen = F 3029 / 3062 6.9 144.4 1.0X -codegen = T hashmap = F 1534 / 1569 13.7 73.2 2.0X -codegen = T hashmap = T 575 / 578 36.5 27.4 5.3X +Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 3475 3516 58 6.0 165.7 1.0X +codegen = T hashmap = F 1939 1958 27 10.8 92.5 1.8X +codegen = T hashmap = T 599 609 9 35.0 28.6 5.8X ================================================================================================ aggregate with multiple key types ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Aggregate w multiple keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -codegen = F 7506 / 7521 2.8 357.9 1.0X -codegen = T hashmap = F 4791 / 4808 4.4 228.5 1.6X -codegen = T hashmap = T 3553 / 3585 5.9 169.4 2.1X +Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 7649 7675 37 2.7 364.7 1.0X +codegen = T hashmap = F 4729 4759 42 4.4 225.5 1.6X +codegen = T hashmap = T 3917 3929 16 5.4 186.8 2.0X ================================================================================================ max function bytecode size of wholestagecodegen ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -codegen = F 608 / 656 1.1 927.1 1.0X -codegen = T hugeMethodLimit = 10000 402 / 419 1.6 613.5 1.5X -codegen = T hugeMethodLimit = 1500 616 / 619 1.1 939.9 1.0X +max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +codegen = F 628 661 25 1.0 958.2 1.0X +codegen = T hugeMethodLimit = 10000 366 385 21 1.8 558.9 1.7X +codegen = T hugeMethodLimit = 1500 620 648 24 1.1 946.7 1.0X ================================================================================================ cube ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -cube wholestage off 3229 / 3237 1.6 615.9 1.0X -cube wholestage on 1285 / 1306 4.1 245.2 2.5X +cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cube wholestage off 3225 3273 68 1.6 615.1 1.0X +cube wholestage on 1636 1680 56 3.2 312.0 2.0X ================================================================================================ hash and BytesToBytesMap ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -UnsafeRowhash 328 / 330 64.0 15.6 1.0X -murmur3 hash 167 / 167 125.4 8.0 2.0X -fast hash 84 / 85 249.0 4.0 3.9X -arrayEqual 192 / 192 109.3 9.1 1.7X -Java HashMap (Long) 144 / 147 145.9 6.9 2.3X -Java HashMap (two ints) 147 / 153 142.3 7.0 2.2X -Java HashMap (UnsafeRow) 785 / 788 26.7 37.4 0.4X -LongToUnsafeRowMap (opt=false) 456 / 457 46.0 21.8 0.7X -LongToUnsafeRowMap (opt=true) 125 / 125 168.3 5.9 2.6X -BytesToBytesMap (off Heap) 885 / 885 23.7 42.2 0.4X -BytesToBytesMap (on Heap) 860 / 864 24.4 41.0 0.4X -Aggregate HashMap 56 / 56 373.9 2.7 5.8X +BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UnsafeRowhash 331 332 0 63.3 15.8 1.0X +murmur3 hash 148 148 0 141.4 7.1 2.2X +fast hash 75 76 6 280.6 3.6 4.4X +arrayEqual 176 176 0 119.0 8.4 1.9X +Java HashMap (Long) 140 144 6 149.7 6.7 2.4X +Java HashMap (two ints) 153 157 5 137.2 7.3 2.2X +Java HashMap (UnsafeRow) 845 852 6 24.8 40.3 0.4X +LongToUnsafeRowMap (opt=false) 463 482 26 45.3 22.1 0.7X +LongToUnsafeRowMap (opt=true) 118 120 6 178.1 5.6 2.8X +BytesToBytesMap (off Heap) 935 937 2 22.4 44.6 0.4X +BytesToBytesMap (on Heap) 897 901 5 23.4 42.8 0.4X +Aggregate HashMap 57 58 0 369.9 2.7 5.8X diff --git a/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..9cd7ad16ac2bb --- /dev/null +++ b/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt @@ -0,0 +1,24 @@ +================================================================================================ +ORC Write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Without bloom filter 17367 17786 592 5.8 173.7 1.0X +With bloom filter 20556 20596 57 4.9 205.6 0.8X + + +================================================================================================ +ORC Read +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Without bloom filter 2204 2226 31 45.4 22.0 1.0X +With bloom filter 1415 1465 71 70.7 14.2 1.6X + + diff --git a/sql/core/benchmarks/BloomFilterBenchmark-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-results.txt index 2eeb26c899b42..b4e3e843798e6 100644 --- a/sql/core/benchmarks/BloomFilterBenchmark-results.txt +++ b/sql/core/benchmarks/BloomFilterBenchmark-results.txt @@ -2,23 +2,23 @@ ORC Write ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Write 100M rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Without bloom filter 16765 / 17587 6.0 167.7 1.0X -With bloom filter 20060 / 20626 5.0 200.6 0.8X +Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Without bloom filter 19489 19693 289 5.1 194.9 1.0X +With bloom filter 23045 23148 145 4.3 230.5 0.8X ================================================================================================ ORC Read ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Read a row from 100M rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Without bloom filter 1857 / 1904 53.9 18.6 1.0X -With bloom filter 1399 / 1437 71.5 14.0 1.3X +Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Without bloom filter 2038 2084 65 49.1 20.4 1.0X +With bloom filter 1465 1475 15 68.3 14.6 1.4X diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk11-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..bc13f6e9b9424 --- /dev/null +++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk11-results.txt @@ -0,0 +1,60 @@ +================================================================================================ +Parquet writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 2552 2690 195 6.2 162.2 1.0X +Output Single Double Column 2865 2892 38 5.5 182.2 0.9X +Output Int and String Column 7876 7885 12 2.0 500.7 0.3X +Output Partitions 5079 5871 1120 3.1 322.9 0.5X +Output Buckets 6980 6994 20 2.3 443.8 0.4X + + +================================================================================================ +ORC writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 1799 1902 146 8.7 114.4 1.0X +Output Single Double Column 2268 2276 11 6.9 144.2 0.8X +Output Int and String Column 6650 6670 28 2.4 422.8 0.3X +Output Partitions 4697 4719 31 3.3 298.6 0.4X +Output Buckets 6394 6436 60 2.5 406.5 0.3X + + +================================================================================================ +JSON writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 2778 3522 1052 5.7 176.6 1.0X +Output Single Double Column 4222 4269 67 3.7 268.4 0.7X +Output Int and String Column 10822 10845 33 1.5 688.0 0.3X +Output Partitions 5450 5523 104 2.9 346.5 0.5X +Output Buckets 10827 11622 1123 1.5 688.4 0.3X + + +================================================================================================ +CSV writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 3649 3698 68 4.3 232.0 1.0X +Output Single Double Column 4612 4696 120 3.4 293.2 0.8X +Output Int and String Column 7334 7517 258 2.1 466.3 0.5X +Output Partitions 6386 6541 220 2.5 406.0 0.6X +Output Buckets 8692 9439 1057 1.8 552.6 0.4X + + diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt index 9d656fc10dce4..1380a6158954a 100644 --- a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt +++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt @@ -2,59 +2,59 @@ Parquet writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Parquet writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 2537 2594 81 6.2 161.3 1.0X +Output Single Double Column 2615 2645 42 6.0 166.3 1.0X +Output Int and String Column 8931 8933 4 1.8 567.8 0.3X +Output Partitions 5011 5321 438 3.1 318.6 0.5X +Output Buckets 6789 6793 6 2.3 431.6 0.4X ================================================================================================ ORC writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +ORC writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 1780 1937 222 8.8 113.2 1.0X +Output Single Double Column 2248 2265 23 7.0 142.9 0.8X +Output Int and String Column 7996 8031 49 2.0 508.4 0.2X +Output Partitions 4827 4939 159 3.3 306.9 0.4X +Output Buckets 6521 6595 104 2.4 414.6 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +JSON writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 2791 2823 45 5.6 177.5 1.0X +Output Single Double Column 4078 5089 1431 3.9 259.3 0.7X +Output Int and String Column 10249 10296 66 1.5 651.6 0.3X +Output Partitions 5247 5296 69 3.0 333.6 0.5X +Output Buckets 8168 9714 2187 1.9 519.3 0.3X ================================================================================================ CSV writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +CSV writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Output Single Int Column 4055 4200 205 3.9 257.8 1.0X +Output Single Double Column 4978 5010 45 3.2 316.5 0.8X +Output Int and String Column 9285 9294 13 1.7 590.3 0.4X +Output Partitions 6681 6738 80 2.4 424.8 0.6X +Output Buckets 9495 10052 787 1.7 603.7 0.4X diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..b65b236fd71f2 --- /dev/null +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -0,0 +1,59 @@ +================================================================================================ +Benchmark to measure CSV read/write performance +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +One quoted string 56894 57106 184 0.0 1137889.9 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Select 1000 columns 220825 222234 2018 0.0 220825.5 1.0X +Select 100 columns 50507 50723 278 0.0 50506.6 4.4X +Select one column 38629 38642 16 0.0 38628.6 5.7X +count() 8549 8597 51 0.1 8549.2 25.8X +Select 100 columns, one bad input field 68309 68474 182 0.0 68309.2 3.2X +Select 100 columns, corrupt record field 74551 74701 136 0.0 74551.5 3.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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() 27745 28050 276 0.4 2774.5 1.0X +Select 1 column + count() 19989 20315 319 0.5 1998.9 1.4X +count() 6091 6109 25 1.6 609.1 4.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Create a dataset of timestamps 2235 2301 59 4.5 223.5 1.0X +to_csv(timestamp) 16033 16205 153 0.6 1603.3 0.1X +write timestamps to files 13556 13685 167 0.7 1355.6 0.2X +Create a dataset of dates 2262 2290 44 4.4 226.2 1.0X +to_csv(date) 11122 11160 33 0.9 1112.2 0.2X +write dates to files 8436 8486 76 1.2 843.6 0.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +read timestamp text from files 2617 2644 26 3.8 261.7 1.0X +read timestamps from files 53245 53381 149 0.2 5324.5 0.0X +infer timestamps from files 103797 104026 257 0.1 10379.7 0.0X +read date text from files 2371 2378 7 4.2 237.1 1.1X +read date from files 41808 41929 177 0.2 4180.8 0.1X +infer date from files 35069 35336 458 0.3 3506.9 0.1X +timestamp strings 3104 3127 21 3.2 310.4 0.8X +parse timestamps from Dataset[String] 61888 62132 342 0.2 6188.8 0.0X +infer timestamps from Dataset[String] 112494 114609 1949 0.1 11249.4 0.0X +date strings 3558 3603 41 2.8 355.8 0.7X +parse dates from Dataset[String] 45871 46000 120 0.2 4587.1 0.1X +from_csv(timestamp) 56975 57035 53 0.2 5697.5 0.0X +from_csv(date) 43711 43795 74 0.2 4371.1 0.1X + + diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index 888c2ce9f2845..d2037e86a3a71 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,58 +2,58 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +One quoted string 62603 62755 133 0.0 1252055.6 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +Select 1000 columns 225032 225919 782 0.0 225031.7 1.0X +Select 100 columns 51982 52290 286 0.0 51982.1 4.3X +Select one column 40167 40283 133 0.0 40167.4 5.6X +count() 11435 11593 176 0.1 11435.1 19.7X +Select 100 columns, one bad input field 66864 66968 174 0.0 66864.1 3.4X +Select 100 columns, corrupt record field 79570 80418 1080 0.0 79569.5 2.8X -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +Select 10 columns + count() 23271 23389 103 0.4 2327.1 1.0X +Select 1 column + count() 18206 19772 NaN 0.5 1820.6 1.3X +count() 8500 8521 18 1.2 850.0 2.7X -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +Create a dataset of timestamps 2025 2068 66 4.9 202.5 1.0X +to_csv(timestamp) 22192 22983 879 0.5 2219.2 0.1X +write timestamps to files 15949 16030 72 0.6 1594.9 0.1X +Create a dataset of dates 2200 2234 32 4.5 220.0 0.9X +to_csv(date) 18268 18341 73 0.5 1826.8 0.1X +write dates to files 10495 10722 214 1.0 1049.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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +read timestamp text from files 6491 6503 18 1.5 649.1 1.0X +read timestamps from files 56069 56795 874 0.2 5606.9 0.1X +infer timestamps from files 113383 114203 825 0.1 11338.3 0.1X +read date text from files 6411 6419 10 1.6 641.1 1.0X +read date from files 46245 46371 138 0.2 4624.5 0.1X +infer date from files 43623 43906 291 0.2 4362.3 0.1X +timestamp strings 4951 4959 7 2.0 495.1 1.3X +parse timestamps from Dataset[String] 65786 66309 663 0.2 6578.6 0.1X +infer timestamps from Dataset[String] 130891 133861 1928 0.1 13089.1 0.0X +date strings 3814 3895 84 2.6 381.4 1.7X +parse dates from Dataset[String] 52259 52960 614 0.2 5225.9 0.1X +from_csv(timestamp) 63013 63306 291 0.2 6301.3 0.1X +from_csv(date) 49840 52352 NaN 0.2 4984.0 0.1X diff --git a/sql/core/benchmarks/ColumnarBatchBenchmark-jdk11-results.txt b/sql/core/benchmarks/ColumnarBatchBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..d5dcdab6b942a --- /dev/null +++ b/sql/core/benchmarks/ColumnarBatchBenchmark-jdk11-results.txt @@ -0,0 +1,59 @@ +================================================================================================ +Int Read/Write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Int Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Java Array 233 234 0 1406.0 0.7 1.0X +ByteBuffer Unsafe 396 397 0 826.5 1.2 0.6X +ByteBuffer API 603 605 3 543.0 1.8 0.4X +DirectByteBuffer 1189 1196 9 275.6 3.6 0.2X +Unsafe Buffer 321 324 3 1022.0 1.0 0.7X +Column(on heap) 235 236 2 1393.1 0.7 1.0X +Column(off heap) 524 526 1 624.8 1.6 0.4X +Column(off heap direct) 317 318 1 1034.1 1.0 0.7X +UnsafeRow (on heap) 690 693 3 474.8 2.1 0.3X +UnsafeRow (off heap) 642 646 3 510.1 2.0 0.4X +Column On Heap Append 522 527 4 628.0 1.6 0.4X + + +================================================================================================ +Boolean Read/Write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Boolean Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Bitset 778 780 3 431.1 2.3 1.0X +Byte Array 355 356 1 944.9 1.1 2.2X + + +================================================================================================ +String Read/Write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +On Heap 447 453 5 36.6 27.3 1.0X +Off Heap 652 658 4 25.1 39.8 0.7X + + +================================================================================================ +Array Vector Read +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Array Vector Read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +On Heap Read Size Only 101 102 0 1616.4 0.6 1.0X +Off Heap Read Size Only 305 307 2 536.8 1.9 0.3X +On Heap Read Elements 4255 4256 1 38.5 26.0 0.0X +Off Heap Read Elements 6097 6104 10 26.9 37.2 0.0X + + diff --git a/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt b/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt index 59637162f0a1d..8ae21d7a7e653 100644 --- a/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt +++ b/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt @@ -2,58 +2,58 @@ Int Read/Write ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Int Read/Write: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Java Array 244 / 244 1342.3 0.7 1.0X -ByteBuffer Unsafe 445 / 445 736.5 1.4 0.5X -ByteBuffer API 2124 / 2125 154.3 6.5 0.1X -DirectByteBuffer 750 / 750 437.2 2.3 0.3X -Unsafe Buffer 234 / 236 1401.3 0.7 1.0X -Column(on heap) 245 / 245 1335.6 0.7 1.0X -Column(off heap) 489 / 489 670.3 1.5 0.5X -Column(off heap direct) 236 / 236 1388.1 0.7 1.0X -UnsafeRow (on heap) 532 / 534 616.0 1.6 0.5X -UnsafeRow (off heap) 564 / 565 580.7 1.7 0.4X -Column On Heap Append 489 / 489 670.6 1.5 0.5X +Int Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Java Array 246 246 0 1333.2 0.8 1.0X +ByteBuffer Unsafe 446 447 1 734.6 1.4 0.6X +ByteBuffer API 2141 2144 4 153.0 6.5 0.1X +DirectByteBuffer 745 745 0 439.9 2.3 0.3X +Unsafe Buffer 237 240 1 1379.7 0.7 1.0X +Column(on heap) 247 247 0 1327.3 0.8 1.0X +Column(off heap) 487 488 1 673.1 1.5 0.5X +Column(off heap direct) 242 243 1 1354.1 0.7 1.0X +UnsafeRow (on heap) 541 543 1 605.4 1.7 0.5X +UnsafeRow (off heap) 581 583 2 563.7 1.8 0.4X +Column On Heap Append 520 521 1 629.8 1.6 0.5X ================================================================================================ Boolean Read/Write ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Boolean Read/Write: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Bitset 879 / 879 381.9 2.6 1.0X -Byte Array 794 / 794 422.6 2.4 1.1X +Boolean Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Bitset 883 883 1 380.1 2.6 1.0X +Byte Array 775 777 2 432.9 2.3 1.1X ================================================================================================ String Read/Write ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String Read/Write: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -On Heap 449 / 449 36.5 27.4 1.0X -Off Heap 679 / 679 24.1 41.4 0.7X +String Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +On Heap 463 470 10 35.4 28.3 1.0X +Off Heap 676 677 2 24.2 41.2 0.7X ================================================================================================ Array Vector Read ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Array Vector Read: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -On Heap Read Size Only 713 / 713 229.8 4.4 1.0X -Off Heap Read Size Only 757 / 757 216.5 4.6 0.9X -On Heap Read Elements 3648 / 3650 44.9 22.3 0.2X -Off Heap Read Elements 5263 / 5265 31.1 32.1 0.1X +Array Vector Read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +On Heap Read Size Only 422 422 0 388.7 2.6 1.0X +Off Heap Read Size Only 259 259 0 633.2 1.6 1.6X +On Heap Read Elements 3932 3960 39 41.7 24.0 0.1X +Off Heap Read Elements 5143 5173 43 31.9 31.4 0.1X diff --git a/sql/core/benchmarks/CompressionSchemeBenchmark-jdk11-results.txt b/sql/core/benchmarks/CompressionSchemeBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..4fd57a9e95560 --- /dev/null +++ b/sql/core/benchmarks/CompressionSchemeBenchmark-jdk11-results.txt @@ -0,0 +1,137 @@ +================================================================================================ +Compression Scheme Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +BOOLEAN Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 3 3 0 21087.3 0.0 1.0X +RunLengthEncoding(2.514) 739 739 1 90.8 11.0 0.0X +BooleanBitSet(0.125) 378 379 1 177.4 5.6 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +BOOLEAN Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 147 147 1 456.1 2.2 1.0X +RunLengthEncoding 731 732 1 91.8 10.9 0.2X +BooleanBitSet 1410 1411 1 47.6 21.0 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 7 7 0 9729.9 0.1 1.0X +RunLengthEncoding(1.491) 1576 1576 1 42.6 23.5 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1151 1152 1 58.3 17.2 1.0X +RunLengthEncoding 1619 1621 3 41.4 24.1 0.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 7 7 0 10135.7 0.1 1.0X +RunLengthEncoding(2.010) 1659 1660 0 40.4 24.7 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1151 1151 1 58.3 17.2 1.0X +RunLengthEncoding 1655 1655 0 40.5 24.7 0.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 23 23 0 2952.8 0.3 1.0X +RunLengthEncoding(0.997) 2356 2356 0 28.5 35.1 0.0X +DictionaryEncoding(0.500) 1402 1402 0 47.9 20.9 0.0X +IntDelta(0.250) 213 213 0 315.2 3.2 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1319 1319 1 50.9 19.7 1.0X +RunLengthEncoding 1803 1806 5 37.2 26.9 0.7X +DictionaryEncoding 931 931 0 72.1 13.9 1.4X +IntDelta 817 821 4 82.2 12.2 1.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 23 23 0 2976.8 0.3 1.0X +RunLengthEncoding(1.337) 2552 2552 1 26.3 38.0 0.0X +DictionaryEncoding(0.501) 1377 1377 0 48.7 20.5 0.0X +IntDelta(0.250) 213 214 2 315.3 3.2 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1438 1439 1 46.7 21.4 1.0X +RunLengthEncoding 1987 1988 1 33.8 29.6 0.7X +DictionaryEncoding 1249 1250 0 53.7 18.6 1.2X +IntDelta 1135 1136 3 59.2 16.9 1.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 45 45 0 1487.9 0.7 1.0X +RunLengthEncoding(0.750) 2496 2496 1 26.9 37.2 0.0X +DictionaryEncoding(0.250) 1433 1433 1 46.8 21.4 0.0X +LongDelta(0.125) 215 215 0 312.6 3.2 0.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1468 1469 1 45.7 21.9 1.0X +RunLengthEncoding 1977 1981 6 33.9 29.5 0.7X +DictionaryEncoding 1248 1250 3 53.8 18.6 1.2X +LongDelta 838 840 2 80.1 12.5 1.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 47 47 0 1437.2 0.7 1.0X +RunLengthEncoding(1.002) 2743 2744 0 24.5 40.9 0.0X +DictionaryEncoding(0.251) 2016 2016 0 33.3 30.0 0.0X +LongDelta(0.125) 215 217 5 312.1 3.2 0.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1468 1468 0 45.7 21.9 1.0X +RunLengthEncoding 2020 2021 2 33.2 30.1 0.7X +DictionaryEncoding 1248 1248 0 53.8 18.6 1.2X +LongDelta 1131 1134 4 59.4 16.8 1.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +STRING Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 71 71 0 939.6 1.1 1.0X +RunLengthEncoding(0.890) 6050 6052 2 11.1 90.2 0.0X +DictionaryEncoding(0.167) 3723 3725 2 18.0 55.5 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +STRING Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 2804 2815 16 23.9 41.8 1.0X +RunLengthEncoding 3390 3391 1 19.8 50.5 0.8X +DictionaryEncoding 2901 2905 5 23.1 43.2 1.0X + + diff --git a/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt b/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt index caa9378301f5d..3f6fbe35a7b86 100644 --- a/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt +++ b/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt @@ -2,136 +2,136 @@ Compression Scheme Benchmark ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -BOOLEAN Encode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 4 / 4 17998.9 0.1 1.0X -RunLengthEncoding(2.501) 680 / 680 98.7 10.1 0.0X -BooleanBitSet(0.125) 365 / 365 183.9 5.4 0.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +BOOLEAN Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 3 3 0 21114.6 0.0 1.0X +RunLengthEncoding(2.505) 694 696 4 96.7 10.3 0.0X +BooleanBitSet(0.125) 366 366 0 183.4 5.5 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -BOOLEAN Decode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 144 / 144 466.5 2.1 1.0X -RunLengthEncoding 679 / 679 98.9 10.1 0.2X -BooleanBitSet 1425 / 1431 47.1 21.2 0.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +BOOLEAN Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 145 145 0 464.2 2.2 1.0X +RunLengthEncoding 735 735 0 91.3 10.9 0.2X +BooleanBitSet 1437 1437 1 46.7 21.4 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SHORT Encode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 7 / 7 10115.0 0.1 1.0X -RunLengthEncoding(1.494) 1671 / 1672 40.2 24.9 0.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 -SHORT Decode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 1128 / 1128 59.5 16.8 1.0X -RunLengthEncoding 1630 / 1633 41.2 24.3 0.7X - -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 -SHORT Encode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 7 / 7 10164.2 0.1 1.0X -RunLengthEncoding(1.989) 1562 / 1563 43.0 23.3 0.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 -SHORT Decode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 1127 / 1127 59.6 16.8 1.0X -RunLengthEncoding 1629 / 1631 41.2 24.3 0.7X - -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 -INT Encode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 22 / 23 2983.2 0.3 1.0X -RunLengthEncoding(1.003) 2426 / 2427 27.7 36.1 0.0X -DictionaryEncoding(0.500) 958 / 958 70.1 14.3 0.0X -IntDelta(0.250) 286 / 286 235.0 4.3 0.1X - -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 -INT Decode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 1268 / 1269 52.9 18.9 1.0X -RunLengthEncoding 1906 / 1911 35.2 28.4 0.7X -DictionaryEncoding 981 / 982 68.4 14.6 1.3X -IntDelta 812 / 817 82.6 12.1 1.6X - -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 -INT Encode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 23 / 23 2926.9 0.3 1.0X -RunLengthEncoding(1.326) 2614 / 2614 25.7 38.9 0.0X -DictionaryEncoding(0.501) 1024 / 1024 65.5 15.3 0.0X -IntDelta(0.250) 286 / 286 234.7 4.3 0.1X - -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 -INT Decode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 1433 / 1433 46.8 21.4 1.0X -RunLengthEncoding 1923 / 1926 34.9 28.6 0.7X -DictionaryEncoding 1285 / 1285 52.2 19.2 1.1X -IntDelta 1129 / 1137 59.4 16.8 1.3X - -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 -LONG Encode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 45 / 45 1495.6 0.7 1.0X -RunLengthEncoding(0.738) 2662 / 2663 25.2 39.7 0.0X -DictionaryEncoding(0.250) 1269 / 1269 52.9 18.9 0.0X -LongDelta(0.125) 450 / 450 149.1 6.7 0.1X - -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 -LONG Decode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 1483 / 1483 45.3 22.1 1.0X -RunLengthEncoding 1875 / 1875 35.8 27.9 0.8X -DictionaryEncoding 1213 / 1214 55.3 18.1 1.2X -LongDelta 816 / 817 82.2 12.2 1.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 -LONG Encode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 45 / 45 1489.3 0.7 1.0X -RunLengthEncoding(1.003) 2906 / 2906 23.1 43.3 0.0X -DictionaryEncoding(0.251) 1610 / 1610 41.7 24.0 0.0X -LongDelta(0.125) 451 / 451 148.7 6.7 0.1X - -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 -LONG Decode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 1485 / 1485 45.2 22.1 1.0X -RunLengthEncoding 1889 / 1890 35.5 28.2 0.8X -DictionaryEncoding 1215 / 1216 55.2 18.1 1.2X -LongDelta 1107 / 1110 60.6 16.5 1.3X - -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 -STRING Encode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough(1.000) 67 / 68 994.5 1.0 1.0X -RunLengthEncoding(0.894) 5877 / 5882 11.4 87.6 0.0X -DictionaryEncoding(0.167) 3597 / 3602 18.7 53.6 0.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 -STRING Decode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -PassThrough 3243 / 3244 20.7 48.3 1.0X -RunLengthEncoding 3598 / 3601 18.7 53.6 0.9X -DictionaryEncoding 3182 / 3182 21.1 47.4 1.0X +SHORT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 7 7 0 9336.6 0.1 1.0X +RunLengthEncoding(1.494) 1912 1917 7 35.1 28.5 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1104 1104 0 60.8 16.4 1.0X +RunLengthEncoding 1627 1628 0 41.2 24.3 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 7 7 0 9710.6 0.1 1.0X +RunLengthEncoding(2.003) 2021 2027 9 33.2 30.1 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SHORT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1104 1104 0 60.8 16.5 1.0X +RunLengthEncoding 1621 1621 0 41.4 24.1 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 24 24 0 2854.3 0.4 1.0X +RunLengthEncoding(1.005) 2395 2396 2 28.0 35.7 0.0X +DictionaryEncoding(0.500) 1366 1366 0 49.1 20.3 0.0X +IntDelta(0.250) 286 287 0 234.2 4.3 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1248 1248 0 53.8 18.6 1.0X +RunLengthEncoding 1738 1739 2 38.6 25.9 0.7X +DictionaryEncoding 969 970 0 69.2 14.4 1.3X +IntDelta 777 779 1 86.3 11.6 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 23 23 0 2874.4 0.3 1.0X +RunLengthEncoding(1.334) 2581 2581 0 26.0 38.5 0.0X +DictionaryEncoding(0.501) 1490 1490 0 45.0 22.2 0.0X +IntDelta(0.250) 286 286 0 234.5 4.3 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +INT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1389 1389 0 48.3 20.7 1.0X +RunLengthEncoding 1903 1903 0 35.3 28.4 0.7X +DictionaryEncoding 1231 1232 1 54.5 18.3 1.1X +IntDelta 1103 1108 7 60.8 16.4 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 48 48 0 1405.2 0.7 1.0X +RunLengthEncoding(0.757) 2525 2525 1 26.6 37.6 0.0X +DictionaryEncoding(0.250) 1380 1381 1 48.6 20.6 0.0X +LongDelta(0.125) 474 474 0 141.7 7.1 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1348 1349 0 49.8 20.1 1.0X +RunLengthEncoding 1850 1851 2 36.3 27.6 0.7X +DictionaryEncoding 1190 1192 3 56.4 17.7 1.1X +LongDelta 801 801 0 83.8 11.9 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 46 46 0 1451.2 0.7 1.0X +RunLengthEncoding(1.003) 2742 2743 1 24.5 40.9 0.0X +DictionaryEncoding(0.251) 1714 1715 0 39.1 25.5 0.0X +LongDelta(0.125) 476 476 0 140.9 7.1 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LONG Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 1362 1363 1 49.3 20.3 1.0X +RunLengthEncoding 1862 1863 1 36.0 27.7 0.7X +DictionaryEncoding 1190 1192 3 56.4 17.7 1.1X +LongDelta 1079 1082 4 62.2 16.1 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +STRING Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough(1.000) 67 67 0 994.8 1.0 1.0X +RunLengthEncoding(0.888) 6135 6137 2 10.9 91.4 0.0X +DictionaryEncoding(0.167) 3747 3748 0 17.9 55.8 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +STRING Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +PassThrough 3180 3185 8 21.1 47.4 1.0X +RunLengthEncoding 3658 3660 3 18.3 54.5 0.9X +DictionaryEncoding 3292 3295 4 20.4 49.1 1.0X diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..55cb301dba174 --- /dev/null +++ b/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt @@ -0,0 +1,252 @@ +================================================================================================ +SQL Single Numeric Column Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 27115 27169 76 0.6 1723.9 1.0X +SQL Json 9061 9124 89 1.7 576.1 3.0X +SQL Parquet Vectorized 196 232 39 80.4 12.4 138.5X +SQL Parquet MR 2187 2216 40 7.2 139.1 12.4X +SQL ORC Vectorized 335 344 5 46.9 21.3 80.9X +SQL ORC MR 1757 1786 42 9.0 111.7 15.4X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 201 205 5 78.3 12.8 1.0X +ParquetReader Vectorized -> Row 91 92 1 173.2 5.8 2.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 27969 27972 4 0.6 1778.2 1.0X +SQL Json 10328 10389 87 1.5 656.6 2.7X +SQL Parquet Vectorized 217 237 24 72.5 13.8 128.8X +SQL Parquet MR 2494 2567 103 6.3 158.6 11.2X +SQL ORC Vectorized 310 321 10 50.8 19.7 90.3X +SQL ORC MR 1901 1907 9 8.3 120.9 14.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 272 280 10 57.8 17.3 1.0X +ParquetReader Vectorized -> Row 144 185 68 109.3 9.1 1.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 29507 29532 34 0.5 1876.0 1.0X +SQL Json 10463 10474 16 1.5 665.2 2.8X +SQL Parquet Vectorized 193 204 10 81.3 12.3 152.6X +SQL Parquet MR 2948 2954 7 5.3 187.5 10.0X +SQL ORC Vectorized 268 277 9 58.7 17.0 110.1X +SQL ORC MR 1910 1950 57 8.2 121.4 15.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 263 278 38 59.7 16.7 1.0X +ParquetReader Vectorized -> Row 259 266 9 60.7 16.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 36696 36771 106 0.4 2333.0 1.0X +SQL Json 13496 13520 34 1.2 858.0 2.7X +SQL Parquet Vectorized 282 292 9 55.7 17.9 130.0X +SQL Parquet MR 3358 3383 36 4.7 213.5 10.9X +SQL ORC Vectorized 409 414 5 38.5 26.0 89.7X +SQL ORC MR 2250 2275 35 7.0 143.1 16.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 360 372 15 43.6 22.9 1.0X +ParquetReader Vectorized -> Row 354 357 5 44.4 22.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 30462 30466 5 0.5 1936.7 1.0X +SQL Json 12916 12948 45 1.2 821.2 2.4X +SQL Parquet Vectorized 181 185 5 86.7 11.5 168.0X +SQL Parquet MR 2810 2820 14 5.6 178.7 10.8X +SQL ORC Vectorized 426 430 4 36.9 27.1 71.6X +SQL ORC MR 2106 2112 9 7.5 133.9 14.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 255 261 7 61.6 16.2 1.0X +ParquetReader Vectorized -> Row 285 288 5 55.1 18.1 0.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 36950 36979 41 0.4 2349.2 1.0X +SQL Json 18794 18795 2 0.8 1194.9 2.0X +SQL Parquet Vectorized 279 295 17 56.3 17.8 132.3X +SQL Parquet MR 3933 4025 130 4.0 250.0 9.4X +SQL ORC Vectorized 521 527 6 30.2 33.2 70.9X +SQL ORC MR 2290 2326 51 6.9 145.6 16.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 356 365 12 44.2 22.6 1.0X +ParquetReader Vectorized -> Row 350 352 2 45.0 22.2 1.0X + + +================================================================================================ +Int and String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 26764 26810 65 0.4 2552.4 1.0X +SQL Json 12107 12195 124 0.9 1154.6 2.2X +SQL Parquet Vectorized 2202 2210 10 4.8 210.0 12.2X +SQL Parquet MR 5297 5302 6 2.0 505.2 5.1X +SQL ORC Vectorized 2356 2372 23 4.5 224.7 11.4X +SQL ORC MR 4370 4419 70 2.4 416.8 6.1X + + +================================================================================================ +Repeated String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 19953 19966 18 0.5 1902.8 1.0X +SQL Json 7151 7220 98 1.5 681.9 2.8X +SQL Parquet Vectorized 692 695 3 15.1 66.0 28.8X +SQL Parquet MR 2859 2943 118 3.7 272.6 7.0X +SQL ORC Vectorized 535 540 5 19.6 51.0 37.3X +SQL ORC MR 2157 2162 8 4.9 205.7 9.3X + + +================================================================================================ +Partitioned Table Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column - CSV 46775 46785 13 0.3 2973.9 1.0X +Data column - Json 13891 13893 2 1.1 883.2 3.4X +Data column - Parquet Vectorized 301 306 7 52.3 19.1 155.6X +Data column - Parquet MR 3565 3572 10 4.4 226.7 13.1X +Data column - ORC Vectorized 434 458 36 36.2 27.6 107.7X +Data column - ORC MR 2337 2354 24 6.7 148.6 20.0X +Partition column - CSV 10645 10688 61 1.5 676.8 4.4X +Partition column - Json 10912 10973 87 1.4 693.7 4.3X +Partition column - Parquet Vectorized 93 103 9 169.4 5.9 503.8X +Partition column - Parquet MR 1588 1597 13 9.9 100.9 29.5X +Partition column - ORC Vectorized 92 99 11 170.7 5.9 507.6X +Partition column - ORC MR 1714 1716 3 9.2 109.0 27.3X +Both columns - CSV 46199 46222 32 0.3 2937.3 1.0X +Both columns - Json 17279 17291 18 0.9 1098.6 2.7X +Both columns - Parquet Vectorized 346 355 13 45.4 22.0 135.0X +Both columns - Parquet MR 3883 3908 35 4.1 246.9 12.0X +Both columns - ORC Vectorized 577 618 57 27.3 36.7 81.1X +Both columns - ORC MR 2967 3024 80 5.3 188.7 15.8X + + +================================================================================================ +String with Nulls Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 23623 23731 153 0.4 2252.9 1.0X +SQL Json 13299 13432 187 0.8 1268.3 1.8X +SQL Parquet Vectorized 1464 1466 4 7.2 139.6 16.1X +SQL Parquet MR 7602 7628 37 1.4 724.9 3.1X +ParquetReader Vectorized 1032 1043 15 10.2 98.4 22.9X +SQL ORC Vectorized 1206 1211 7 8.7 115.0 19.6X +SQL ORC MR 4726 4991 374 2.2 450.7 5.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 23715 24152 619 0.4 2261.6 1.0X +SQL Json 10120 10280 226 1.0 965.1 2.3X +SQL Parquet Vectorized 1063 1072 13 9.9 101.4 22.3X +SQL Parquet MR 5460 5464 5 1.9 520.8 4.3X +ParquetReader Vectorized 934 936 4 11.2 89.0 25.4X +SQL ORC Vectorized 1094 1094 0 9.6 104.3 21.7X +SQL ORC MR 3964 4401 618 2.6 378.0 6.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 21348 21472 175 0.5 2035.9 1.0X +SQL Json 5877 5956 112 1.8 560.5 3.6X +SQL Parquet Vectorized 244 256 22 43.0 23.2 87.6X +SQL Parquet MR 3139 3371 328 3.3 299.4 6.8X +ParquetReader Vectorized 238 245 9 44.1 22.7 89.7X +SQL ORC Vectorized 378 383 7 27.7 36.0 56.5X +SQL ORC MR 2234 2315 115 4.7 213.0 9.6X + + +================================================================================================ +Single Column Scan From Wide Columns +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 4053 4064 16 0.3 3865.4 1.0X +SQL Json 4115 4118 4 0.3 3924.6 1.0X +SQL Parquet Vectorized 72 82 11 14.5 69.0 56.0X +SQL Parquet MR 314 325 18 3.3 299.3 12.9X +SQL ORC Vectorized 80 86 8 13.1 76.2 50.7X +SQL ORC MR 250 253 2 4.2 238.5 16.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 7802 7849 66 0.1 7440.8 1.0X +SQL Json 16640 17481 1190 0.1 15868.8 0.5X +SQL Parquet Vectorized 106 126 31 9.9 101.0 73.7X +SQL Parquet MR 349 358 7 3.0 332.8 22.4X +SQL ORC Vectorized 108 115 10 9.7 102.7 72.5X +SQL ORC MR 284 298 20 3.7 270.5 27.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 12639 12672 47 0.1 12053.5 1.0X +SQL Json 30613 30688 106 0.0 29194.8 0.4X +SQL Parquet Vectorized 145 165 21 7.2 138.3 87.2X +SQL Parquet MR 384 393 9 2.7 366.4 32.9X +SQL ORC Vectorized 129 134 5 8.1 123.2 97.8X +SQL ORC MR 280 319 66 3.7 266.9 45.2X + + diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt index efb0601cc5d65..4e8bb019fba43 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -6,109 +6,109 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 23939 24126 265 0.7 1522.0 1.0X -SQL Json 8908 9008 142 1.8 566.4 2.7X -SQL Parquet Vectorized 192 229 36 82.1 12.2 125.0X -SQL Parquet MR 2356 2363 10 6.7 149.8 10.2X -SQL ORC Vectorized 329 347 25 47.9 20.9 72.9X -SQL ORC MR 1711 1747 50 9.2 108.8 14.0X +SQL CSV 23037 23172 191 0.7 1464.7 1.0X +SQL Json 8682 8686 5 1.8 552.0 2.7X +SQL Parquet Vectorized 183 205 32 85.9 11.6 125.8X +SQL Parquet MR 2189 2200 15 7.2 139.2 10.5X +SQL ORC Vectorized 296 306 5 53.1 18.8 77.7X +SQL ORC MR 1705 1717 18 9.2 108.4 13.5X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 194 197 4 81.1 12.3 1.0X -ParquetReader Vectorized -> Row 97 102 13 162.3 6.2 2.0X +ParquetReader Vectorized 195 200 7 80.9 12.4 1.0X +ParquetReader Vectorized -> Row 96 97 2 163.0 6.1 2.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 24603 24607 6 0.6 1564.2 1.0X -SQL Json 9587 9652 92 1.6 609.5 2.6X -SQL Parquet Vectorized 227 241 13 69.4 14.4 108.6X -SQL Parquet MR 2432 2441 12 6.5 154.6 10.1X -SQL ORC Vectorized 320 327 8 49.2 20.3 76.9X -SQL ORC MR 1889 1921 46 8.3 120.1 13.0X +SQL CSV 25126 25265 196 0.6 1597.5 1.0X +SQL Json 9442 9445 4 1.7 600.3 2.7X +SQL Parquet Vectorized 228 240 7 69.1 14.5 110.4X +SQL Parquet MR 2432 2445 19 6.5 154.6 10.3X +SQL ORC Vectorized 315 319 6 49.9 20.0 79.8X +SQL ORC MR 1901 1916 21 8.3 120.9 13.2X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 290 294 8 54.3 18.4 1.0X -ParquetReader Vectorized -> Row 252 256 5 62.4 16.0 1.2X +ParquetReader Vectorized 293 302 9 53.6 18.7 1.0X +ParquetReader Vectorized -> Row 264 266 2 59.7 16.8 1.1X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 26742 26743 1 0.6 1700.2 1.0X -SQL Json 10855 10855 0 1.4 690.1 2.5X -SQL Parquet Vectorized 195 202 7 80.8 12.4 137.3X -SQL Parquet MR 2805 2806 0 5.6 178.4 9.5X -SQL ORC Vectorized 376 383 5 41.8 23.9 71.1X -SQL ORC MR 2021 2092 102 7.8 128.5 13.2X +SQL CSV 27419 27443 34 0.6 1743.3 1.0X +SQL Json 9831 9836 8 1.6 625.0 2.8X +SQL Parquet Vectorized 192 198 9 81.8 12.2 142.7X +SQL Parquet MR 2696 2740 62 5.8 171.4 10.2X +SQL ORC Vectorized 329 335 8 47.9 20.9 83.4X +SQL ORC MR 1932 2006 105 8.1 122.8 14.2X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 248 253 5 63.4 15.8 1.0X -ParquetReader Vectorized -> Row 249 251 2 63.1 15.9 1.0X +ParquetReader Vectorized 248 253 6 63.5 15.8 1.0X +ParquetReader Vectorized -> Row 250 256 7 62.9 15.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 34841 34855 20 0.5 2215.1 1.0X -SQL Json 14121 14133 18 1.1 897.8 2.5X -SQL Parquet Vectorized 288 303 17 54.7 18.3 121.2X -SQL Parquet MR 3178 3197 27 4.9 202.0 11.0X -SQL ORC Vectorized 465 476 8 33.8 29.6 74.9X -SQL ORC MR 2255 2260 6 7.0 143.4 15.4X +SQL CSV 34898 34907 14 0.5 2218.7 1.0X +SQL Json 12760 12764 5 1.2 811.3 2.7X +SQL Parquet Vectorized 283 289 5 55.6 18.0 123.3X +SQL Parquet MR 3238 3240 3 4.9 205.9 10.8X +SQL ORC Vectorized 401 405 7 39.2 25.5 87.0X +SQL ORC MR 2274 2290 23 6.9 144.6 15.3X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 344 354 11 45.8 21.8 1.0X -ParquetReader Vectorized -> Row 383 385 3 41.1 24.3 0.9X +ParquetReader Vectorized 339 351 16 46.5 21.5 1.0X +ParquetReader Vectorized -> Row 342 348 13 46.0 21.8 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 29336 29563 322 0.5 1865.1 1.0X -SQL Json 13452 13544 130 1.2 855.3 2.2X -SQL Parquet Vectorized 186 200 22 84.8 11.8 158.1X -SQL Parquet MR 2752 2815 90 5.7 175.0 10.7X -SQL ORC Vectorized 460 465 6 34.2 29.3 63.7X -SQL ORC MR 2054 2072 26 7.7 130.6 14.3X +SQL CSV 28872 28886 20 0.5 1835.6 1.0X +SQL Json 13360 13377 24 1.2 849.4 2.2X +SQL Parquet Vectorized 181 185 6 86.8 11.5 159.3X +SQL Parquet MR 2645 2651 8 5.9 168.2 10.9X +SQL ORC Vectorized 456 459 5 34.5 29.0 63.4X +SQL ORC MR 2047 2066 26 7.7 130.2 14.1X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 244 246 4 64.6 15.5 1.0X -ParquetReader Vectorized -> Row 247 250 4 63.7 15.7 1.0X +ParquetReader Vectorized 240 246 10 65.5 15.3 1.0X +ParquetReader Vectorized -> Row 245 246 2 64.2 15.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 37812 37897 120 0.4 2404.0 1.0X -SQL Json 19499 19509 15 0.8 1239.7 1.9X -SQL Parquet Vectorized 284 292 10 55.4 18.1 133.2X -SQL Parquet MR 3236 3248 17 4.9 205.7 11.7X -SQL ORC Vectorized 542 558 18 29.0 34.4 69.8X -SQL ORC MR 2273 2298 36 6.9 144.5 16.6X +SQL CSV 36298 36305 10 0.4 2307.7 1.0X +SQL Json 18250 18276 36 0.9 1160.3 2.0X +SQL Parquet Vectorized 278 285 7 56.5 17.7 130.4X +SQL Parquet MR 3144 3146 4 5.0 199.9 11.5X +SQL ORC Vectorized 533 546 16 29.5 33.9 68.1X +SQL ORC MR 2265 2302 53 6.9 144.0 16.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 342 352 13 46.0 21.7 1.0X -ParquetReader Vectorized -> Row 341 344 3 46.1 21.7 1.0X +ParquetReader Vectorized 338 346 12 46.6 21.5 1.0X +ParquetReader Vectorized -> Row 338 344 9 46.5 21.5 1.0X ================================================================================================ @@ -119,12 +119,12 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 26777 26806 41 0.4 2553.7 1.0X -SQL Json 13894 14071 251 0.8 1325.0 1.9X -SQL Parquet Vectorized 2351 2404 75 4.5 224.2 11.4X -SQL Parquet MR 5198 5219 29 2.0 495.8 5.2X -SQL ORC Vectorized 2434 2435 1 4.3 232.1 11.0X -SQL ORC MR 4281 4345 91 2.4 408.3 6.3X +SQL CSV 24839 25273 613 0.4 2368.9 1.0X +SQL Json 11861 11869 11 0.9 1131.2 2.1X +SQL Parquet Vectorized 2298 2305 9 4.6 219.2 10.8X +SQL Parquet MR 5045 5053 10 2.1 481.2 4.9X +SQL ORC Vectorized 2391 2405 21 4.4 228.0 10.4X +SQL ORC MR 4561 4645 118 2.3 435.0 5.4X ================================================================================================ @@ -135,12 +135,12 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 15779 16507 1029 0.7 1504.8 1.0X -SQL Json 7866 7877 14 1.3 750.2 2.0X -SQL Parquet Vectorized 820 826 5 12.8 78.2 19.2X -SQL Parquet MR 2646 2658 17 4.0 252.4 6.0X -SQL ORC Vectorized 638 644 7 16.4 60.9 24.7X -SQL ORC MR 2205 2222 25 4.8 210.3 7.2X +SQL CSV 14147 14244 137 0.7 1349.1 1.0X +SQL Json 7289 7306 23 1.4 695.1 1.9X +SQL Parquet Vectorized 818 821 4 12.8 78.0 17.3X +SQL Parquet MR 2562 2570 11 4.1 244.4 5.5X +SQL ORC Vectorized 571 579 8 18.3 54.5 24.8X +SQL ORC MR 2143 2164 31 4.9 204.3 6.6X ================================================================================================ @@ -151,24 +151,24 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column - CSV 38142 38183 58 0.4 2425.0 1.0X -Data column - Json 14664 14667 4 1.1 932.3 2.6X -Data column - Parquet Vectorized 304 318 13 51.8 19.3 125.7X -Data column - Parquet MR 3378 3384 8 4.7 214.8 11.3X -Data column - ORC Vectorized 475 481 7 33.1 30.2 80.3X -Data column - ORC MR 2324 2356 46 6.8 147.7 16.4X -Partition column - CSV 14680 14742 88 1.1 933.3 2.6X -Partition column - Json 11200 11251 73 1.4 712.1 3.4X -Partition column - Parquet Vectorized 102 111 14 154.7 6.5 375.1X -Partition column - Parquet MR 1477 1483 9 10.7 93.9 25.8X -Partition column - ORC Vectorized 100 112 18 157.4 6.4 381.6X -Partition column - ORC MR 1675 1685 15 9.4 106.5 22.8X -Both columns - CSV 41925 41929 6 0.4 2665.5 0.9X -Both columns - Json 15409 15422 18 1.0 979.7 2.5X -Both columns - Parquet Vectorized 351 358 10 44.8 22.3 108.7X -Both columns - Parquet MR 3719 3720 2 4.2 236.4 10.3X -Both columns - ORC Vectorized 609 630 23 25.8 38.7 62.6X -Both columns - ORC MR 2959 2959 1 5.3 188.1 12.9X +Data column - CSV 38652 38680 40 0.4 2457.4 1.0X +Data column - Json 12756 12760 5 1.2 811.0 3.0X +Data column - Parquet Vectorized 304 314 9 51.7 19.3 127.2X +Data column - Parquet MR 3387 3393 9 4.6 215.3 11.4X +Data column - ORC Vectorized 425 436 10 37.0 27.0 91.0X +Data column - ORC MR 2303 2330 38 6.8 146.4 16.8X +Partition column - CSV 11239 11249 14 1.4 714.5 3.4X +Partition column - Json 10477 10479 3 1.5 666.1 3.7X +Partition column - Parquet Vectorized 95 102 9 165.5 6.0 406.7X +Partition column - Parquet MR 1574 1575 1 10.0 100.1 24.6X +Partition column - ORC Vectorized 95 106 20 166.3 6.0 408.5X +Partition column - ORC MR 1682 1693 15 9.4 106.9 23.0X +Both columns - CSV 39146 39203 81 0.4 2488.8 1.0X +Both columns - Json 14675 14691 23 1.1 933.0 2.6X +Both columns - Parquet Vectorized 347 351 3 45.3 22.1 111.4X +Both columns - Parquet MR 3680 3717 52 4.3 234.0 10.5X +Both columns - ORC Vectorized 556 565 8 28.3 35.3 69.6X +Both columns - ORC MR 2909 2923 20 5.4 184.9 13.3X ================================================================================================ @@ -179,37 +179,37 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 19510 19709 282 0.5 1860.6 1.0X -SQL Json 11816 11822 8 0.9 1126.9 1.7X -SQL Parquet Vectorized 1535 1548 18 6.8 146.4 12.7X -SQL Parquet MR 5491 5514 33 1.9 523.6 3.6X -ParquetReader Vectorized 1126 1129 5 9.3 107.4 17.3X -SQL ORC Vectorized 1200 1215 21 8.7 114.5 16.3X -SQL ORC MR 3901 3904 4 2.7 372.1 5.0X +SQL CSV 17457 17740 401 0.6 1664.9 1.0X +SQL Json 12276 12287 16 0.9 1170.7 1.4X +SQL Parquet Vectorized 1525 1539 20 6.9 145.4 11.5X +SQL Parquet MR 5051 5098 66 2.1 481.7 3.5X +ParquetReader Vectorized 1115 1123 12 9.4 106.3 15.7X +SQL ORC Vectorized 1269 1294 37 8.3 121.0 13.8X +SQL ORC MR 3938 3951 17 2.7 375.6 4.4X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 21439 21457 26 0.5 2044.6 1.0X -SQL Json 9653 9669 22 1.1 920.6 2.2X -SQL Parquet Vectorized 1126 1131 8 9.3 107.4 19.0X -SQL Parquet MR 3947 3961 19 2.7 376.4 5.4X -ParquetReader Vectorized 998 1023 36 10.5 95.2 21.5X -SQL ORC Vectorized 1274 1277 4 8.2 121.5 16.8X -SQL ORC MR 3424 3425 1 3.1 326.5 6.3X +SQL CSV 18086 18119 47 0.6 1724.8 1.0X +SQL Json 8484 8851 520 1.2 809.1 2.1X +SQL Parquet Vectorized 1127 1131 5 9.3 107.5 16.0X +SQL Parquet MR 4120 4131 15 2.5 392.9 4.4X +ParquetReader Vectorized 984 1019 49 10.7 93.9 18.4X +SQL ORC Vectorized 1208 1211 4 8.7 115.2 15.0X +SQL ORC MR 3401 3410 13 3.1 324.4 5.3X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 17885 17893 11 0.6 1705.7 1.0X -SQL Json 5201 5210 13 2.0 496.0 3.4X -SQL Parquet Vectorized 261 267 6 40.2 24.9 68.6X -SQL Parquet MR 2841 2853 18 3.7 270.9 6.3X -ParquetReader Vectorized 244 246 3 43.1 23.2 73.4X -SQL ORC Vectorized 465 468 1 22.5 44.4 38.4X -SQL ORC MR 1904 1945 58 5.5 181.6 9.4X +SQL CSV 24825 24970 205 0.4 2367.5 1.0X +SQL Json 9847 9857 14 1.1 939.1 2.5X +SQL Parquet Vectorized 258 261 6 40.7 24.6 96.3X +SQL Parquet MR 3182 3242 85 3.3 303.4 7.8X +ParquetReader Vectorized 241 242 2 43.6 22.9 103.2X +SQL ORC Vectorized 453 456 4 23.1 43.2 54.8X +SQL ORC MR 1917 1927 13 5.5 182.8 12.9X ================================================================================================ @@ -220,33 +220,33 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 3841 3861 28 0.3 3663.1 1.0X -SQL Json 3780 3787 10 0.3 3604.6 1.0X -SQL Parquet Vectorized 83 90 10 12.7 79.0 46.4X -SQL Parquet MR 291 303 18 3.6 277.9 13.2X -SQL ORC Vectorized 93 106 20 11.3 88.8 41.2X -SQL ORC MR 217 224 10 4.8 206.6 17.7X +SQL CSV 5163 5174 16 0.2 4923.5 1.0X +SQL Json 4459 4538 111 0.2 4252.7 1.2X +SQL Parquet Vectorized 78 84 8 13.4 74.7 65.9X +SQL Parquet MR 511 519 9 2.1 486.9 10.1X +SQL ORC Vectorized 86 93 11 12.2 82.1 60.0X +SQL ORC MR 350 359 7 3.0 333.6 14.8X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 8896 8971 106 0.1 8483.9 1.0X -SQL Json 14731 14773 59 0.1 14048.2 0.6X -SQL Parquet Vectorized 120 146 26 8.8 114.0 74.4X -SQL Parquet MR 330 363 33 3.2 314.4 27.0X -SQL ORC Vectorized 122 130 11 8.6 115.9 73.2X -SQL ORC MR 248 254 9 4.2 237.0 35.8X +SQL CSV 9839 9842 4 0.1 9383.4 1.0X +SQL Json 15887 15889 4 0.1 15150.7 0.6X +SQL Parquet Vectorized 115 125 11 9.1 109.9 85.4X +SQL Parquet MR 666 671 8 1.6 635.4 14.8X +SQL ORC Vectorized 115 120 6 9.1 110.1 85.2X +SQL ORC MR 455 458 3 2.3 433.7 21.6X OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 14771 14817 65 0.1 14086.3 1.0X -SQL Json 29677 29787 157 0.0 28302.0 0.5X -SQL Parquet Vectorized 182 191 13 5.8 173.8 81.1X -SQL Parquet MR 1209 1213 5 0.9 1153.1 12.2X -SQL ORC Vectorized 165 176 17 6.3 157.7 89.3X -SQL ORC MR 809 813 4 1.3 771.4 18.3X +SQL CSV 15858 15891 46 0.1 15123.5 1.0X +SQL Json 30200 30256 80 0.0 28800.6 0.5X +SQL Parquet Vectorized 160 165 7 6.5 153.0 98.8X +SQL Parquet MR 682 690 7 1.5 650.3 23.3X +SQL ORC Vectorized 143 150 10 7.4 136.0 111.2X +SQL ORC MR 494 509 15 2.1 471.4 32.1X diff --git a/sql/core/benchmarks/DatasetBenchmark-jdk11-results.txt b/sql/core/benchmarks/DatasetBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..f07fae9121df1 --- /dev/null +++ b/sql/core/benchmarks/DatasetBenchmark-jdk11-results.txt @@ -0,0 +1,46 @@ +================================================================================================ +Dataset Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +back-to-back map long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 14574 14759 261 6.9 145.7 1.0X +DataFrame 2468 2655 264 40.5 24.7 5.9X +Dataset 3498 3533 50 28.6 35.0 4.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +back-to-back map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 17877 18133 361 5.6 178.8 1.0X +DataFrame 5968 5991 33 16.8 59.7 3.0X +Dataset 12638 12859 313 7.9 126.4 1.4X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +back-to-back filter Long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 3399 3464 92 29.4 34.0 1.0X +DataFrame 1609 1628 28 62.2 16.1 2.1X +Dataset 3637 3648 16 27.5 36.4 0.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +back-to-back filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 4850 4859 13 20.6 48.5 1.0X +DataFrame 211 244 21 472.9 2.1 22.9X +Dataset 5864 6126 372 17.1 58.6 0.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +aggregate: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD sum 4821 4914 131 20.7 48.2 1.0X +DataFrame sum 71 83 8 1412.4 0.7 68.1X +Dataset sum using Aggregator 6001 6012 16 16.7 60.0 0.8X +Dataset complex Aggregator 10247 10455 294 9.8 102.5 0.5X + + diff --git a/sql/core/benchmarks/DatasetBenchmark-results.txt b/sql/core/benchmarks/DatasetBenchmark-results.txt index dcc190eb45c03..2774f14664230 100644 --- a/sql/core/benchmarks/DatasetBenchmark-results.txt +++ b/sql/core/benchmarks/DatasetBenchmark-results.txt @@ -2,45 +2,45 @@ Dataset Benchmark ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -back-to-back map long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -RDD 11800 / 12042 8.5 118.0 1.0X -DataFrame 1927 / 2189 51.9 19.3 6.1X -Dataset 2483 / 2605 40.3 24.8 4.8X +back-to-back map long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 12720 12777 80 7.9 127.2 1.0X +DataFrame 2242 2501 366 44.6 22.4 5.7X +Dataset 3040 3174 189 32.9 30.4 4.2X -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -back-to-back map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -RDD 16286 / 16301 6.1 162.9 1.0X -DataFrame 8101 / 8104 12.3 81.0 2.0X -Dataset 17445 / 17811 5.7 174.4 0.9X +back-to-back map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 15865 15922 82 6.3 158.6 1.0X +DataFrame 8423 8476 75 11.9 84.2 1.9X +Dataset 17180 18142 1361 5.8 171.8 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -back-to-back filter Long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -RDD 2971 / 3184 33.7 29.7 1.0X -DataFrame 1243 / 1296 80.5 12.4 2.4X -Dataset 3062 / 3091 32.7 30.6 1.0X +back-to-back filter Long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 2928 3009 114 34.1 29.3 1.0X +DataFrame 1386 1427 59 72.2 13.9 2.1X +Dataset 3448 3451 5 29.0 34.5 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -back-to-back filter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -RDD 5253 / 5269 19.0 52.5 1.0X -DataFrame 211 / 234 473.4 2.1 24.9X -Dataset 9550 / 9552 10.5 95.5 0.6X +back-to-back filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD 5476 5483 10 18.3 54.8 1.0X +DataFrame 209 235 23 479.1 2.1 26.2X +Dataset 9433 9549 163 10.6 94.3 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -aggregate: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -RDD sum 5086 / 5108 19.7 50.9 1.0X -DataFrame sum 65 / 73 1548.9 0.6 78.8X -Dataset sum using Aggregator 9024 / 9320 11.1 90.2 0.6X -Dataset complex Aggregator 15079 / 15171 6.6 150.8 0.3X +aggregate: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +RDD sum 5146 5239 132 19.4 51.5 1.0X +DataFrame sum 84 99 15 1196.9 0.8 61.6X +Dataset sum using Aggregator 8944 9021 109 11.2 89.4 0.6X +Dataset complex Aggregator 12832 13141 436 7.8 128.3 0.4X diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..27cb1008fa4a1 --- /dev/null +++ b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt @@ -0,0 +1,429 @@ +================================================================================================ +Extract components +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to timestamp wholestage off 460 486 36 21.7 46.0 1.0X +cast to timestamp wholestage on 412 455 70 24.3 41.2 1.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +year of timestamp wholestage off 1351 1351 1 7.4 135.1 1.0X +year of timestamp wholestage on 1277 1312 47 7.8 127.7 1.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +quarter of timestamp wholestage off 1591 1604 19 6.3 159.1 1.0X +quarter of timestamp wholestage on 1583 1600 12 6.3 158.3 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +month of timestamp wholestage off 1263 1268 7 7.9 126.3 1.0X +month of timestamp wholestage on 1281 1316 46 7.8 128.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +weekofyear of timestamp wholestage off 1756 1757 1 5.7 175.6 1.0X +weekofyear of timestamp wholestage on 1748 1762 15 5.7 174.8 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +day of timestamp wholestage off 1250 1253 4 8.0 125.0 1.0X +day of timestamp wholestage on 1270 1285 13 7.9 127.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +dayofyear of timestamp wholestage off 1295 1303 12 7.7 129.5 1.0X +dayofyear of timestamp wholestage on 1294 1308 16 7.7 129.4 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +dayofmonth of timestamp wholestage off 1277 1311 48 7.8 127.7 1.0X +dayofmonth of timestamp wholestage on 1270 1288 24 7.9 127.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +dayofweek of timestamp wholestage off 1472 1472 0 6.8 147.2 1.0X +dayofweek of timestamp wholestage on 1434 1437 4 7.0 143.4 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +weekday of timestamp wholestage off 1363 1372 13 7.3 136.3 1.0X +weekday of timestamp wholestage on 1362 1368 3 7.3 136.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +hour of timestamp wholestage off 748 753 7 13.4 74.8 1.0X +hour of timestamp wholestage on 749 756 9 13.4 74.9 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +minute of timestamp wholestage off 722 725 4 13.9 72.2 1.0X +minute of timestamp wholestage on 750 754 4 13.3 75.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +second of timestamp wholestage off 614 620 8 16.3 61.4 1.0X +second of timestamp wholestage on 611 617 5 16.4 61.1 1.0X + + +================================================================================================ +Current date and time +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +current_date wholestage off 291 298 10 34.4 29.1 1.0X +current_date wholestage on 303 316 13 33.0 30.3 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +current_timestamp wholestage off 328 355 39 30.5 32.8 1.0X +current_timestamp wholestage on 309 497 387 32.4 30.9 1.1X + + +================================================================================================ +Date arithmetic +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to date wholestage off 1094 1103 12 9.1 109.4 1.0X +cast to date wholestage on 1105 1111 5 9.0 110.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +last_day wholestage off 1258 1258 0 7.9 125.8 1.0X +last_day wholestage on 1279 1289 9 7.8 127.9 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +next_day wholestage off 1148 1152 7 8.7 114.8 1.0X +next_day wholestage on 1148 1152 4 8.7 114.8 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_add wholestage off 1069 1074 6 9.4 106.9 1.0X +date_add wholestage on 1087 1091 4 9.2 108.7 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_sub wholestage off 1076 1076 1 9.3 107.6 1.0X +date_sub wholestage on 1086 1094 7 9.2 108.6 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +add_months wholestage off 1442 1442 1 6.9 144.2 1.0X +add_months wholestage on 1406 1426 12 7.1 140.6 1.0X + + +================================================================================================ +Formatting dates +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +format date wholestage off 5082 5179 138 2.0 508.2 1.0X +format date wholestage on 4899 4904 6 2.0 489.9 1.0X + + +================================================================================================ +Formatting timestamps +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +from_unixtime wholestage off 7148 7170 31 1.4 714.8 1.0X +from_unixtime wholestage on 7228 7235 9 1.4 722.8 1.0X + + +================================================================================================ +Convert timestamps +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +from_utc_timestamp wholestage off 982 990 11 10.2 98.2 1.0X +from_utc_timestamp wholestage on 1053 1058 5 9.5 105.3 0.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_utc_timestamp wholestage off 1063 1075 16 9.4 106.3 1.0X +to_utc_timestamp wholestage on 1121 1136 15 8.9 112.1 0.9X + + +================================================================================================ +Intervals +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast interval wholestage off 356 374 25 28.1 35.6 1.0X +cast interval wholestage on 341 353 14 29.3 34.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +datediff wholestage off 1883 1898 22 5.3 188.3 1.0X +datediff wholestage on 1854 1862 8 5.4 185.4 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +months_between wholestage off 1989 1996 10 5.0 198.9 1.0X +months_between wholestage on 2006 2041 36 5.0 200.6 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +window wholestage off 2761 2820 83 0.4 2761.1 1.0X +window wholestage on 47075 47156 75 0.0 47075.0 0.1X + + +================================================================================================ +Truncation +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc YEAR wholestage off 916 919 5 10.9 91.6 1.0X +date_trunc YEAR wholestage on 902 904 2 11.1 90.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc YYYY wholestage off 909 915 9 11.0 90.9 1.0X +date_trunc YYYY wholestage on 902 909 5 11.1 90.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc YY wholestage off 916 925 12 10.9 91.6 1.0X +date_trunc YY wholestage on 903 908 4 11.1 90.3 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MON wholestage off 921 928 11 10.9 92.1 1.0X +date_trunc MON wholestage on 907 912 4 11.0 90.7 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MONTH wholestage off 923 928 7 10.8 92.3 1.0X +date_trunc MONTH wholestage on 902 911 9 11.1 90.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MM wholestage off 922 924 3 10.8 92.2 1.0X +date_trunc MM wholestage on 908 914 7 11.0 90.8 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc DAY wholestage off 644 665 30 15.5 64.4 1.0X +date_trunc DAY wholestage on 632 636 7 15.8 63.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc DD wholestage off 626 636 14 16.0 62.6 1.0X +date_trunc DD wholestage on 635 640 5 15.8 63.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc HOUR wholestage off 652 656 5 15.3 65.2 1.0X +date_trunc HOUR wholestage on 644 648 5 15.5 64.4 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MINUTE wholestage off 617 618 2 16.2 61.7 1.0X +date_trunc MINUTE wholestage on 621 625 4 16.1 62.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc SECOND wholestage off 631 631 0 15.8 63.1 1.0X +date_trunc SECOND wholestage on 625 630 8 16.0 62.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc WEEK wholestage off 760 761 1 13.2 76.0 1.0X +date_trunc WEEK wholestage on 760 766 5 13.2 76.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc QUARTER wholestage off 1656 1657 1 6.0 165.6 1.0X +date_trunc QUARTER wholestage on 1648 1678 36 6.1 164.8 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc year wholestage off 309 309 1 32.4 30.9 1.0X +trunc year wholestage on 310 315 6 32.3 31.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc yyyy wholestage off 310 310 1 32.3 31.0 1.0X +trunc yyyy wholestage on 310 313 2 32.3 31.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc yy wholestage off 307 308 2 32.6 30.7 1.0X +trunc yy wholestage on 310 313 3 32.2 31.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc mon wholestage off 308 317 12 32.5 30.8 1.0X +trunc mon wholestage on 309 316 7 32.3 30.9 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc month wholestage off 306 307 2 32.7 30.6 1.0X +trunc month wholestage on 309 314 6 32.4 30.9 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc mm wholestage off 308 308 0 32.5 30.8 1.0X +trunc mm wholestage on 309 325 11 32.4 30.9 1.0X + + +================================================================================================ +Parsing +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to timestamp str wholestage off 170 170 1 5.9 170.0 1.0X +to timestamp str wholestage on 160 165 4 6.2 160.2 1.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_timestamp wholestage off 1727 1730 4 0.6 1727.1 1.0X +to_timestamp wholestage on 1775 1793 19 0.6 1774.9 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_unix_timestamp wholestage off 1760 1765 6 0.6 1760.4 1.0X +to_unix_timestamp wholestage on 1756 1763 9 0.6 1756.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to date str wholestage off 159 160 0 6.3 159.4 1.0X +to date str wholestage on 156 162 7 6.4 156.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_date wholestage off 2509 2511 3 0.4 2509.2 1.0X +to_date wholestage on 2500 2514 9 0.4 2499.8 1.0X + + +================================================================================================ +Conversion from/to external types +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +From java.sql.Timestamp 353 356 5 14.2 70.5 1.0X +Collect longs 1561 1864 476 3.2 312.2 0.2X +Collect timestamps 2010 2182 149 2.5 402.0 0.2X + + diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 7d562544dd498..32578f6573dfd 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -2,424 +2,428 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to timestamp wholestage off 408 434 37 24.5 40.8 1.0X +cast to timestamp wholestage on 363 372 6 27.6 36.3 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +year of timestamp wholestage off 1317 1327 14 7.6 131.7 1.0X +year of timestamp wholestage on 1258 1308 79 8.0 125.8 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +quarter of timestamp wholestage off 1464 1490 37 6.8 146.4 1.0X +quarter of timestamp wholestage on 1389 1404 13 7.2 138.9 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +month of timestamp wholestage off 1239 1245 7 8.1 123.9 1.0X +month of timestamp wholestage on 1241 1253 8 8.1 124.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +weekofyear of timestamp wholestage off 1896 1899 4 5.3 189.6 1.0X +weekofyear of timestamp wholestage on 1876 1885 6 5.3 187.6 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +day of timestamp wholestage off 1234 1235 1 8.1 123.4 1.0X +day of timestamp wholestage on 1221 1231 8 8.2 122.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +dayofyear of timestamp wholestage off 1269 1275 8 7.9 126.9 1.0X +dayofyear of timestamp wholestage on 1266 1277 9 7.9 126.6 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +dayofmonth of timestamp wholestage off 1236 1238 2 8.1 123.6 1.0X +dayofmonth of timestamp wholestage on 1223 1235 10 8.2 122.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +dayofweek of timestamp wholestage off 1408 1409 2 7.1 140.8 1.0X +dayofweek of timestamp wholestage on 1393 1418 38 7.2 139.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +weekday of timestamp wholestage off 1339 1348 12 7.5 133.9 1.0X +weekday of timestamp wholestage on 1329 1341 22 7.5 132.9 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +hour of timestamp wholestage off 359 366 10 27.9 35.9 1.0X +hour of timestamp wholestage on 344 354 11 29.1 34.4 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +minute of timestamp wholestage off 343 343 0 29.1 34.3 1.0X +minute of timestamp wholestage on 336 339 6 29.8 33.6 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +second of timestamp wholestage off 367 383 23 27.2 36.7 1.0X +second of timestamp wholestage on 335 341 6 29.8 33.5 1.1X ================================================================================================ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +current_date wholestage off 284 286 2 35.2 28.4 1.0X +current_date wholestage on 286 297 18 35.0 28.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 -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +current_timestamp wholestage off 291 336 65 34.4 29.1 1.0X +current_timestamp wholestage on 279 290 11 35.9 27.9 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to date wholestage off 1069 1069 0 9.4 106.9 1.0X +cast to date wholestage on 1037 1043 7 9.6 103.7 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +last_day wholestage off 1275 1287 18 7.8 127.5 1.0X +last_day wholestage on 1257 1267 10 8.0 125.7 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +next_day wholestage off 1114 1116 2 9.0 111.4 1.0X +next_day wholestage on 1077 1081 7 9.3 107.7 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_add wholestage off 1056 1058 2 9.5 105.6 1.0X +date_add wholestage on 1043 1047 3 9.6 104.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_sub wholestage off 1045 1049 5 9.6 104.5 1.0X +date_sub wholestage on 1044 1052 16 9.6 104.4 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +add_months wholestage off 1379 1384 7 7.3 137.9 1.0X +add_months wholestage on 1356 1364 6 7.4 135.6 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 4787 / 4839 2.1 478.7 1.0X -format date wholestage on 4736 / 4802 2.1 473.6 1.0X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +format date wholestage off 5436 5538 144 1.8 543.6 1.0X +format date wholestage on 5283 5295 16 1.9 528.3 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +from_unixtime wholestage off 8980 8998 26 1.1 898.0 1.0X +from_unixtime wholestage on 8921 8939 19 1.1 892.1 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +from_utc_timestamp wholestage off 726 736 15 13.8 72.6 1.0X +from_utc_timestamp wholestage on 701 710 16 14.3 70.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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_utc_timestamp wholestage off 1234 1240 8 8.1 123.4 1.0X +to_utc_timestamp wholestage on 757 770 15 13.2 75.7 1.6X ================================================================================================ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast interval wholestage off 327 328 2 30.6 32.7 1.0X +cast interval wholestage on 299 307 13 33.5 29.9 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +datediff wholestage off 1865 1872 10 5.4 186.5 1.0X +datediff wholestage on 1815 1821 7 5.5 181.5 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +months_between wholestage off 1524 1525 1 6.6 152.4 1.0X +months_between wholestage on 1511 1518 11 6.6 151.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +window wholestage off 2441 2519 111 0.4 2440.7 1.0X +window wholestage on 46149 46165 16 0.0 46149.1 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc YEAR wholestage off 773 777 6 12.9 77.3 1.0X +date_trunc YEAR wholestage on 720 733 21 13.9 72.0 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc YYYY wholestage off 753 754 1 13.3 75.3 1.0X +date_trunc YYYY wholestage on 721 725 3 13.9 72.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc YY wholestage off 759 762 4 13.2 75.9 1.0X +date_trunc YY wholestage on 727 731 3 13.8 72.7 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MON wholestage off 728 731 4 13.7 72.8 1.0X +date_trunc MON wholestage on 717 728 17 13.9 71.7 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MONTH wholestage off 728 729 1 13.7 72.8 1.0X +date_trunc MONTH wholestage on 716 719 2 14.0 71.6 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MM wholestage off 725 732 9 13.8 72.5 1.0X +date_trunc MM wholestage on 721 728 6 13.9 72.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc DAY wholestage off 515 520 7 19.4 51.5 1.0X +date_trunc DAY wholestage on 475 484 8 21.1 47.5 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc DD wholestage off 515 520 7 19.4 51.5 1.0X +date_trunc DD wholestage on 471 484 9 21.2 47.1 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc HOUR wholestage off 504 504 1 19.9 50.4 1.0X +date_trunc HOUR wholestage on 465 471 5 21.5 46.5 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc MINUTE wholestage off 485 488 4 20.6 48.5 1.0X +date_trunc MINUTE wholestage on 452 458 6 22.1 45.2 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc SECOND wholestage off 483 484 1 20.7 48.3 1.0X +date_trunc SECOND wholestage on 447 453 5 22.4 44.7 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc WEEK wholestage off 615 617 2 16.3 61.5 1.0X +date_trunc WEEK wholestage on 576 586 11 17.4 57.6 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +date_trunc QUARTER wholestage off 1512 1513 1 6.6 151.2 1.0X +date_trunc QUARTER wholestage on 1482 1494 11 6.7 148.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc year wholestage off 342 343 1 29.2 34.2 1.0X +trunc year wholestage on 301 304 2 33.3 30.1 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc yyyy wholestage off 321 323 2 31.1 32.1 1.0X +trunc yyyy wholestage on 303 305 2 33.0 30.3 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc yy wholestage off 314 325 15 31.8 31.4 1.0X +trunc yy wholestage on 299 307 10 33.4 29.9 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc mon wholestage off 311 312 2 32.2 31.1 1.0X +trunc mon wholestage on 300 307 6 33.3 30.0 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc month wholestage off 312 314 2 32.1 31.2 1.0X +trunc month wholestage on 299 300 1 33.5 29.9 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +trunc mm wholestage off 308 316 11 32.5 30.8 1.0X +trunc mm wholestage on 298 301 2 33.6 29.8 1.0X ================================================================================================ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to timestamp str wholestage off 228 232 5 4.4 228.3 1.0X +to timestamp str wholestage on 213 214 2 4.7 212.9 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1308 1353 64 0.8 1307.9 1.0X -to_timestamp wholestage on 1197 1230 21 0.8 1197.0 1.1X +to_timestamp wholestage off 1850 1850 0 0.5 1850.0 1.0X +to_timestamp wholestage on 1877 1899 27 0.5 1876.8 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1221 1224 4 0.8 1221.0 1.0X -to_unix_timestamp wholestage on 1224 1228 4 0.8 1223.8 1.0X +to_unix_timestamp wholestage off 1883 1886 4 0.5 1882.7 1.0X +to_unix_timestamp wholestage on 1817 1831 17 0.6 1817.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 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to date str wholestage off 212 212 1 4.7 211.8 1.0X +to date str wholestage on 206 208 2 4.8 206.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_date wholestage off 2334 2335 0 0.4 2334.3 1.0X +to_date wholestage on 2224 2236 13 0.4 2224.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 -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 +================================================================================================ +Conversion from/to external types +================================================================================================ -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +From java.sql.Timestamp 271 274 3 18.4 54.2 1.0X +Collect longs 1386 1925 681 3.6 277.1 0.2X +Collect timestamps 1961 2169 331 2.5 392.2 0.1X diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..4f50a894e5c07 --- /dev/null +++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk11-results.txt @@ -0,0 +1,45 @@ +================================================================================================ +WITHOUT SPILL +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ArrayBuffer 5371 5392 30 19.1 52.5 1.0X +ExternalAppendOnlyUnsafeRowArray 6724 6778 77 15.2 65.7 0.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ArrayBuffer 9120 9121 1 28.7 34.8 1.0X +ExternalAppendOnlyUnsafeRowArray 37713 37739 38 7.0 143.9 0.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ArrayBuffer 18037 18052 21 27.3 36.7 1.0X +ExternalAppendOnlyUnsafeRowArray 34726 34771 63 14.2 70.7 0.5X + + +================================================================================================ +WITH SPILL +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UnsafeExternalSorter 29668 29676 11 8.8 113.2 1.0X +ExternalAppendOnlyUnsafeRowArray 12658 12663 6 20.7 48.3 2.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UnsafeExternalSorter 12 12 0 13.8 72.7 1.0X +ExternalAppendOnlyUnsafeRowArray 8 8 0 19.8 50.6 1.4X + + diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt index 02c6b72f32216..c4be80af1334b 100644 --- a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt +++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt @@ -2,44 +2,44 @@ WITHOUT SPILL ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Array with 100000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ArrayBuffer 7626 7776 212 13.4 74.5 1.0X +ExternalAppendOnlyUnsafeRowArray 8432 8498 93 12.1 82.3 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Array with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ArrayBuffer 11753 11763 15 22.3 44.8 1.0X +ExternalAppendOnlyUnsafeRowArray 38921 39236 446 6.7 148.5 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Array with 30000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ArrayBuffer 23820 23829 12 20.6 48.5 1.0X +ExternalAppendOnlyUnsafeRowArray 33449 33466 25 14.7 68.1 0.7X ================================================================================================ WITH SPILL ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Spilling with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UnsafeExternalSorter 31384 31479 135 8.4 119.7 1.0X +ExternalAppendOnlyUnsafeRowArray 15278 15303 35 17.2 58.3 2.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Spilling with 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UnsafeExternalSorter 11 11 1 14.7 68.0 1.0X +ExternalAppendOnlyUnsafeRowArray 9 10 1 17.1 58.5 1.2X diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt index 9ee9176bea508..74f01f6ee460a 100644 --- a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt @@ -2,99 +2,99 @@ OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 438 548 128 22.8 43.8 1.0X -MILLENNIUM of timestamp 1343 1453 139 7.4 134.3 0.3X -CENTURY of timestamp 1287 1305 16 7.8 128.7 0.3X -DECADE of timestamp 1253 1258 7 8.0 125.3 0.3X -YEAR of timestamp 1224 1247 24 8.2 122.4 0.4X -ISOYEAR of timestamp 1356 1383 35 7.4 135.6 0.3X -QUARTER of timestamp 1386 1395 8 7.2 138.6 0.3X -MONTH of timestamp 1215 1227 11 8.2 121.5 0.4X -WEEK of timestamp 1711 1720 9 5.8 171.1 0.3X -DAY of timestamp 1227 1251 37 8.1 122.7 0.4X -DAYOFWEEK of timestamp 1386 1392 11 7.2 138.6 0.3X -DOW of timestamp 1405 1426 34 7.1 140.5 0.3X -ISODOW of timestamp 1344 1363 30 7.4 134.4 0.3X -DOY of timestamp 1249 1251 3 8.0 124.9 0.4X -HOUR of timestamp 766 773 9 13.1 76.6 0.6X -MINUTE of timestamp 761 774 22 13.1 76.1 0.6X -SECOND of timestamp 627 638 11 16.0 62.7 0.7X -MILLISECONDS of timestamp 700 704 4 14.3 70.0 0.6X -MICROSECONDS of timestamp 615 627 10 16.3 61.5 0.7X -EPOCH of timestamp 28897 28929 29 0.3 2889.7 0.0X +cast to timestamp 342 456 120 29.2 34.2 1.0X +MILLENNIUM of timestamp 1462 1498 53 6.8 146.2 0.2X +CENTURY of timestamp 1229 1293 86 8.1 122.9 0.3X +DECADE of timestamp 1204 1242 56 8.3 120.4 0.3X +YEAR of timestamp 1180 1226 72 8.5 118.0 0.3X +ISOYEAR of timestamp 1279 1301 33 7.8 127.9 0.3X +QUARTER of timestamp 1357 1379 22 7.4 135.7 0.3X +MONTH of timestamp 1155 1182 24 8.7 115.5 0.3X +WEEK of timestamp 1634 1641 10 6.1 163.4 0.2X +DAY of timestamp 1167 1171 6 8.6 116.7 0.3X +DAYOFWEEK of timestamp 1335 1344 8 7.5 133.5 0.3X +DOW of timestamp 1344 1352 11 7.4 134.4 0.3X +ISODOW of timestamp 1279 1281 2 7.8 127.9 0.3X +DOY of timestamp 1172 1183 16 8.5 117.2 0.3X +HOUR of timestamp 722 731 12 13.8 72.2 0.5X +MINUTE of timestamp 725 734 9 13.8 72.5 0.5X +SECOND of timestamp 656 661 6 15.3 65.6 0.5X +MILLISECONDS of timestamp 630 637 9 15.9 63.0 0.5X +MICROSECONDS of timestamp 587 592 7 17.0 58.7 0.6X +EPOCH of timestamp 1036 1047 13 9.7 103.6 0.3X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1078 1081 4 9.3 107.8 1.0X -MILLENNIUM of date 1232 1244 16 8.1 123.2 0.9X -CENTURY of date 1233 1234 1 8.1 123.3 0.9X -DECADE of date 1210 1212 3 8.3 121.0 0.9X -YEAR of date 1201 1212 9 8.3 120.1 0.9X -ISOYEAR of date 1468 1474 5 6.8 146.8 0.7X -QUARTER of date 1474 1482 11 6.8 147.4 0.7X -MONTH of date 1211 1215 4 8.3 121.1 0.9X -WEEK of date 1684 1685 2 5.9 168.4 0.6X -DAY of date 1208 1214 6 8.3 120.8 0.9X -DAYOFWEEK of date 1374 1387 23 7.3 137.4 0.8X -DOW of date 1396 1404 11 7.2 139.6 0.8X -ISODOW of date 1320 1322 3 7.6 132.0 0.8X -DOY of date 1243 1258 13 8.0 124.3 0.9X -HOUR of date 1997 2018 29 5.0 199.7 0.5X -MINUTE of date 2021 2039 26 4.9 202.1 0.5X -SECOND of date 1862 1878 22 5.4 186.2 0.6X -MILLISECONDS of date 1998 2015 16 5.0 199.8 0.5X -MICROSECONDS of date 1893 1901 7 5.3 189.3 0.6X -EPOCH of date 30353 30376 41 0.3 3035.3 0.0X +cast to date 1044 1081 63 9.6 104.4 1.0X +MILLENNIUM of date 1367 1375 10 7.3 136.7 0.8X +CENTURY of date 1181 1183 3 8.5 118.1 0.9X +DECADE of date 1154 1163 8 8.7 115.4 0.9X +YEAR of date 1138 1154 16 8.8 113.8 0.9X +ISOYEAR of date 1394 1401 8 7.2 139.4 0.7X +QUARTER of date 1336 1362 26 7.5 133.6 0.8X +MONTH of date 1137 1144 11 8.8 113.7 0.9X +WEEK of date 1619 1632 18 6.2 161.9 0.6X +DAY of date 1144 1158 22 8.7 114.4 0.9X +DAYOFWEEK of date 1328 1330 2 7.5 132.8 0.8X +DOW of date 1336 1339 4 7.5 133.6 0.8X +ISODOW of date 1272 1276 3 7.9 127.2 0.8X +DOY of date 1170 1174 4 8.6 117.0 0.9X +HOUR of date 1963 1986 35 5.1 196.3 0.5X +MINUTE of date 1969 1984 19 5.1 196.9 0.5X +SECOND of date 1926 1929 3 5.2 192.6 0.5X +MILLISECONDS of date 1928 1939 16 5.2 192.8 0.5X +MICROSECONDS of date 1816 1821 4 5.5 181.6 0.6X +EPOCH of date 2331 2336 4 4.3 233.1 0.4X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 384 389 4 26.1 38.4 1.0X -MILLENNIUM of timestamp 1237 1244 6 8.1 123.7 0.3X -CENTURY of timestamp 1236 1244 7 8.1 123.6 0.3X -DECADE of timestamp 1204 1210 9 8.3 120.4 0.3X -YEAR of timestamp 1197 1207 16 8.4 119.7 0.3X -ISOYEAR of timestamp 1466 1470 4 6.8 146.6 0.3X -QUARTER of timestamp 1500 1505 6 6.7 150.0 0.3X -MONTH of timestamp 1190 1218 25 8.4 119.0 0.3X -WEEK of timestamp 1681 1710 25 5.9 168.1 0.2X -DAY of timestamp 1201 1206 7 8.3 120.1 0.3X -DAYOFWEEK of timestamp 1376 1390 13 7.3 137.6 0.3X -DOW of timestamp 1399 1409 17 7.1 139.9 0.3X -ISODOW of timestamp 1347 1354 8 7.4 134.7 0.3X -DOY of timestamp 1257 1263 6 8.0 125.7 0.3X -HOUR of timestamp 749 753 5 13.4 74.9 0.5X -MINUTE of timestamp 746 749 4 13.4 74.6 0.5X -SECOND of timestamp 626 637 15 16.0 62.6 0.6X -MILLISECONDS of timestamp 695 724 25 14.4 69.5 0.6X -MICROSECONDS of timestamp 611 629 27 16.4 61.1 0.6X -EPOCH of timestamp 28908 28938 31 0.3 2890.8 0.0X +cast to timestamp 315 320 8 31.8 31.5 1.0X +MILLENNIUM of timestamp 1365 1369 4 7.3 136.5 0.2X +CENTURY of timestamp 1176 1179 4 8.5 117.6 0.3X +DECADE of timestamp 1153 1168 14 8.7 115.3 0.3X +YEAR of timestamp 1131 1135 5 8.8 113.1 0.3X +ISOYEAR of timestamp 1388 1397 8 7.2 138.8 0.2X +QUARTER of timestamp 1344 1363 25 7.4 134.4 0.2X +MONTH of timestamp 1135 1141 6 8.8 113.5 0.3X +WEEK of timestamp 1612 1615 5 6.2 161.2 0.2X +DAY of timestamp 1138 1145 7 8.8 113.8 0.3X +DAYOFWEEK of timestamp 1322 1330 7 7.6 132.2 0.2X +DOW of timestamp 1317 1323 5 7.6 131.7 0.2X +ISODOW of timestamp 1284 1287 4 7.8 128.4 0.2X +DOY of timestamp 1163 1172 11 8.6 116.3 0.3X +HOUR of timestamp 709 709 1 14.1 70.9 0.4X +MINUTE of timestamp 713 714 1 14.0 71.3 0.4X +SECOND of timestamp 635 641 6 15.7 63.5 0.5X +MILLISECONDS of timestamp 625 629 7 16.0 62.5 0.5X +MICROSECONDS of timestamp 572 573 2 17.5 57.2 0.6X +EPOCH of timestamp 1031 1033 3 9.7 103.1 0.3X OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1076 1083 6 9.3 107.6 1.0X -MILLENNIUM of date 1230 1236 7 8.1 123.0 0.9X -CENTURY of date 1245 1250 5 8.0 124.5 0.9X -DECADE of date 1206 1211 8 8.3 120.6 0.9X -YEAR of date 1194 1201 6 8.4 119.4 0.9X -ISOYEAR of date 1461 1471 9 6.8 146.1 0.7X -QUARTER of date 1496 1500 7 6.7 149.6 0.7X -MONTH of date 1192 1195 4 8.4 119.2 0.9X -WEEK of date 1682 1687 6 5.9 168.2 0.6X -DAY of date 1199 1207 14 8.3 119.9 0.9X -DAYOFWEEK of date 1372 1383 19 7.3 137.2 0.8X -DOW of date 1384 1393 14 7.2 138.4 0.8X -ISODOW of date 1327 1338 10 7.5 132.7 0.8X -DOY of date 1243 1247 7 8.0 124.3 0.9X -HOUR of date 2001 2010 10 5.0 200.1 0.5X -MINUTE of date 2046 2053 9 4.9 204.6 0.5X -SECOND of date 1859 1863 4 5.4 185.9 0.6X -MILLISECONDS of date 2000 2013 16 5.0 200.0 0.5X -MICROSECONDS of date 1856 1857 1 5.4 185.6 0.6X -EPOCH of date 30365 30388 29 0.3 3036.5 0.0X +cast to date 994 998 3 10.1 99.4 1.0X +MILLENNIUM of date 1361 1371 9 7.3 136.1 0.7X +CENTURY of date 1171 1180 15 8.5 117.1 0.8X +DECADE of date 1149 1153 4 8.7 114.9 0.9X +YEAR of date 1128 1130 2 8.9 112.8 0.9X +ISOYEAR of date 1384 1398 19 7.2 138.4 0.7X +QUARTER of date 1333 1345 19 7.5 133.3 0.7X +MONTH of date 1132 1138 7 8.8 113.2 0.9X +WEEK of date 1616 1624 9 6.2 161.6 0.6X +DAY of date 1132 1135 3 8.8 113.2 0.9X +DAYOFWEEK of date 1320 1322 3 7.6 132.0 0.8X +DOW of date 1317 1319 2 7.6 131.7 0.8X +ISODOW of date 1269 1272 5 7.9 126.9 0.8X +DOY of date 1171 1190 17 8.5 117.1 0.8X +HOUR of date 1960 1962 2 5.1 196.0 0.5X +MINUTE of date 1974 1975 1 5.1 197.4 0.5X +SECOND of date 1915 1922 8 5.2 191.5 0.5X +MILLISECONDS of date 1923 1924 2 5.2 192.3 0.5X +MICROSECONDS of date 1817 1819 1 5.5 181.7 0.5X +EPOCH of date 2323 2326 2 4.3 232.3 0.4X diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 2e1df667bbe8c..89d3616d2c205 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,100 +1,119 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 254 275 35 39.4 25.4 1.0X -MILLENNIUM of timestamp 1149 1159 9 8.7 114.9 0.2X -CENTURY of timestamp 1102 1115 16 9.1 110.2 0.2X -DECADE of timestamp 1024 1036 11 9.8 102.4 0.2X -YEAR of timestamp 1000 1004 5 10.0 100.0 0.3X -ISOYEAR of timestamp 1090 1101 11 9.2 109.0 0.2X -QUARTER of timestamp 1169 1178 7 8.6 116.9 0.2X -MONTH of timestamp 981 984 4 10.2 98.1 0.3X -WEEK of timestamp 1372 1388 14 7.3 137.2 0.2X -DAY of timestamp 994 1000 7 10.1 99.4 0.3X -DAYOFWEEK of timestamp 1102 1108 6 9.1 110.2 0.2X -DOW of timestamp 1102 1105 3 9.1 110.2 0.2X -ISODOW of timestamp 1063 1078 18 9.4 106.3 0.2X -DOY of timestamp 1015 1021 5 9.9 101.5 0.2X -HOUR of timestamp 385 390 5 26.0 38.5 0.7X -MINUTE of timestamp 387 391 7 25.8 38.7 0.7X -SECOND of timestamp 381 382 1 26.2 38.1 0.7X -MILLISECONDS of timestamp 584 588 4 17.1 58.4 0.4X -MICROSECONDS of timestamp 482 500 16 20.8 48.2 0.5X -EPOCH of timestamp 957 961 6 10.4 95.7 0.3X +cast to timestamp 404 460 71 24.8 40.4 1.0X +MILLENNIUM of timestamp 1432 1580 128 7.0 143.2 0.3X +CENTURY of timestamp 1380 1390 10 7.2 138.0 0.3X +DECADE of timestamp 1254 1261 8 8.0 125.4 0.3X +YEAR of timestamp 1229 1236 9 8.1 122.9 0.3X +ISOYEAR of timestamp 1382 1415 44 7.2 138.2 0.3X +QUARTER of timestamp 1444 1458 16 6.9 144.4 0.3X +MONTH of timestamp 1212 1257 60 8.2 121.2 0.3X +WEEK of timestamp 1718 1730 12 5.8 171.8 0.2X +DAY of timestamp 1180 1201 22 8.5 118.0 0.3X +DAYOFWEEK of timestamp 1371 1427 68 7.3 137.1 0.3X +DOW of timestamp 1365 1378 18 7.3 136.5 0.3X +ISODOW of timestamp 1312 1319 7 7.6 131.2 0.3X +DOY of timestamp 1224 1238 16 8.2 122.4 0.3X +HOUR of timestamp 343 355 16 29.2 34.3 1.2X +MINUTE of timestamp 341 344 3 29.3 34.1 1.2X +SECOND of timestamp 553 556 2 18.1 55.3 0.7X +MILLISECONDS of timestamp 562 565 3 17.8 56.2 0.7X +MICROSECONDS of timestamp 435 465 49 23.0 43.5 0.9X +EPOCH of timestamp 1211 1217 9 8.3 121.1 0.3X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 834 842 12 12.0 83.4 1.0X -MILLENNIUM of date 1078 1088 16 9.3 107.8 0.8X -CENTURY of date 1063 1067 4 9.4 106.3 0.8X -DECADE of date 989 992 3 10.1 98.9 0.8X -YEAR of date 975 976 1 10.3 97.5 0.9X -ISOYEAR of date 1177 1186 9 8.5 117.7 0.7X -QUARTER of date 1219 1222 4 8.2 121.9 0.7X -MONTH of date 982 992 10 10.2 98.2 0.8X -WEEK of date 1360 1364 6 7.4 136.0 0.6X -DAY of date 973 980 7 10.3 97.3 0.9X -DAYOFWEEK of date 1100 1104 7 9.1 110.0 0.8X -DOW of date 1091 1096 4 9.2 109.1 0.8X -ISODOW of date 1053 1057 6 9.5 105.3 0.8X -DOY of date 1006 1012 4 9.9 100.6 0.8X -HOUR of date 1683 1688 6 5.9 168.3 0.5X -MINUTE of date 1686 1691 5 5.9 168.6 0.5X -SECOND of date 1706 1714 7 5.9 170.6 0.5X -MILLISECONDS of date 1881 1887 6 5.3 188.1 0.4X -MICROSECONDS of date 1767 1778 16 5.7 176.7 0.5X -EPOCH of date 2274 2281 7 4.4 227.4 0.4X +cast to date 1030 1038 8 9.7 103.0 1.0X +MILLENNIUM of date 1302 1306 6 7.7 130.2 0.8X +CENTURY of date 1309 1315 6 7.6 130.9 0.8X +DECADE of date 1197 1202 7 8.4 119.7 0.9X +YEAR of date 1182 1192 10 8.5 118.2 0.9X +ISOYEAR of date 1421 1429 10 7.0 142.1 0.7X +QUARTER of date 1454 1468 14 6.9 145.4 0.7X +MONTH of date 1217 1230 19 8.2 121.7 0.8X +WEEK of date 1723 1725 3 5.8 172.3 0.6X +DAY of date 1188 1191 4 8.4 118.8 0.9X +DAYOFWEEK of date 1348 1355 6 7.4 134.8 0.8X +DOW of date 1344 1353 8 7.4 134.4 0.8X +ISODOW of date 1301 1311 9 7.7 130.1 0.8X +DOY of date 1237 1241 4 8.1 123.7 0.8X +HOUR of date 1465 1474 16 6.8 146.5 0.7X +MINUTE of date 1463 1471 8 6.8 146.3 0.7X +SECOND of date 1720 1725 5 5.8 172.0 0.6X +MILLISECONDS of date 1739 1749 9 5.8 173.9 0.6X +MICROSECONDS of date 1607 1611 4 6.2 160.7 0.6X +EPOCH of date 2370 2377 10 4.2 237.0 0.4X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 212 214 3 47.2 21.2 1.0X -MILLENNIUM of timestamp 1082 1092 10 9.2 108.2 0.2X -CENTURY of timestamp 1087 1089 2 9.2 108.7 0.2X -DECADE of timestamp 986 1001 21 10.1 98.6 0.2X -YEAR of timestamp 983 985 2 10.2 98.3 0.2X -ISOYEAR of timestamp 1160 1168 12 8.6 116.0 0.2X -QUARTER of timestamp 1219 1228 9 8.2 121.9 0.2X -MONTH of timestamp 973 979 5 10.3 97.3 0.2X -WEEK of timestamp 1348 1357 8 7.4 134.8 0.2X -DAY of timestamp 969 974 5 10.3 96.9 0.2X -DAYOFWEEK of timestamp 1093 1096 3 9.1 109.3 0.2X -DOW of timestamp 1089 1096 7 9.2 108.9 0.2X -ISODOW of timestamp 1048 1056 7 9.5 104.8 0.2X -DOY of timestamp 1007 1011 5 9.9 100.7 0.2X -HOUR of timestamp 385 387 3 26.0 38.5 0.6X -MINUTE of timestamp 382 384 2 26.2 38.2 0.6X -SECOND of timestamp 377 378 1 26.5 37.7 0.6X -MILLISECONDS of timestamp 572 574 2 17.5 57.2 0.4X -MICROSECONDS of timestamp 473 490 18 21.1 47.3 0.4X -EPOCH of timestamp 937 947 11 10.7 93.7 0.2X +cast to timestamp 307 312 6 32.6 30.7 1.0X +MILLENNIUM of timestamp 1294 1298 4 7.7 129.4 0.2X +CENTURY of timestamp 1317 1321 6 7.6 131.7 0.2X +DECADE of timestamp 1202 1207 8 8.3 120.2 0.3X +YEAR of timestamp 1196 1201 8 8.4 119.6 0.3X +ISOYEAR of timestamp 1431 1440 10 7.0 143.1 0.2X +QUARTER of timestamp 1483 1487 5 6.7 148.3 0.2X +MONTH of timestamp 1205 1209 4 8.3 120.5 0.3X +WEEK of timestamp 1728 1731 4 5.8 172.8 0.2X +DAY of timestamp 1190 1193 3 8.4 119.0 0.3X +DAYOFWEEK of timestamp 1369 1371 2 7.3 136.9 0.2X +DOW of timestamp 1371 1373 3 7.3 137.1 0.2X +ISODOW of timestamp 1316 1321 8 7.6 131.6 0.2X +DOY of timestamp 1236 1241 6 8.1 123.6 0.2X +HOUR of timestamp 342 343 1 29.3 34.2 0.9X +MINUTE of timestamp 338 352 23 29.6 33.8 0.9X +SECOND of timestamp 555 556 1 18.0 55.5 0.6X +MILLISECONDS of timestamp 570 573 4 17.6 57.0 0.5X +MICROSECONDS of timestamp 436 442 5 22.9 43.6 0.7X +EPOCH of timestamp 1212 1218 7 8.3 121.2 0.3X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 835 836 1 12.0 83.5 1.0X -MILLENNIUM of date 1062 1070 10 9.4 106.2 0.8X -CENTURY of date 1060 1068 12 9.4 106.0 0.8X -DECADE of date 984 992 8 10.2 98.4 0.8X -YEAR of date 971 973 3 10.3 97.1 0.9X -ISOYEAR of date 1158 1175 22 8.6 115.8 0.7X -QUARTER of date 1239 1239 0 8.1 123.9 0.7X -MONTH of date 972 987 23 10.3 97.2 0.9X -WEEK of date 1346 1350 5 7.4 134.6 0.6X -DAY of date 970 971 1 10.3 97.0 0.9X -DAYOFWEEK of date 1099 1107 10 9.1 109.9 0.8X -DOW of date 1089 1091 2 9.2 108.9 0.8X -ISODOW of date 1053 1062 13 9.5 105.3 0.8X -DOY of date 1008 1010 4 9.9 100.8 0.8X -HOUR of date 1697 1703 7 5.9 169.7 0.5X -MINUTE of date 1697 1699 3 5.9 169.7 0.5X -SECOND of date 1682 1692 8 5.9 168.2 0.5X -MILLISECONDS of date 1884 1890 7 5.3 188.4 0.4X -MICROSECONDS of date 1765 1770 7 5.7 176.5 0.5X -EPOCH of date 2273 2282 8 4.4 227.3 0.4X +cast to date 1017 1025 13 9.8 101.7 1.0X +MILLENNIUM of date 1314 1318 5 7.6 131.4 0.8X +CENTURY of date 1318 1334 16 7.6 131.8 0.8X +DECADE of date 1205 1210 5 8.3 120.5 0.8X +YEAR of date 1187 1190 3 8.4 118.7 0.9X +ISOYEAR of date 1409 1432 21 7.1 140.9 0.7X +QUARTER of date 1477 1482 6 6.8 147.7 0.7X +MONTH of date 1193 1199 7 8.4 119.3 0.9X +WEEK of date 1711 1719 7 5.8 171.1 0.6X +DAY of date 1183 1192 7 8.5 118.3 0.9X +DAYOFWEEK of date 1368 1371 6 7.3 136.8 0.7X +DOW of date 1361 1366 8 7.3 136.1 0.7X +ISODOW of date 1312 1315 3 7.6 131.2 0.8X +DOY of date 1230 1233 2 8.1 123.0 0.8X +HOUR of date 1480 1483 2 6.8 148.0 0.7X +MINUTE of date 1473 1489 27 6.8 147.3 0.7X +SECOND of date 1731 1737 7 5.8 173.1 0.6X +MILLISECONDS of date 1744 1749 6 5.7 174.4 0.6X +MICROSECONDS of date 1592 1594 1 6.3 159.2 0.6X +EPOCH of date 2368 2371 3 4.2 236.8 0.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.15 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to interval 1365 1395 31 7.3 136.5 1.0X +MILLENNIUM of interval 1620 1651 27 6.2 162.0 0.8X +CENTURY of interval 1469 1487 22 6.8 146.9 0.9X +DECADE of interval 1462 1473 17 6.8 146.2 0.9X +YEAR of interval 1438 1447 8 7.0 143.8 0.9X +QUARTER of interval 1456 1458 3 6.9 145.6 0.9X +MONTH of interval 1440 1452 16 6.9 144.0 0.9X +DAY of interval 1478 1485 6 6.8 147.8 0.9X +HOUR of interval 1579 1580 3 6.3 157.9 0.9X +MINUTE of interval 1598 1605 11 6.3 159.8 0.9X +SECOND of interval 1571 1579 10 6.4 157.1 0.9X +MILLISECONDS of interval 1570 1577 6 6.4 157.0 0.9X +MICROSECONDS of interval 1484 1488 5 6.7 148.4 0.9X +EPOCH of interval 1521 1522 1 6.6 152.1 0.9X diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk11-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..e01561364b989 --- /dev/null +++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk11-results.txt @@ -0,0 +1,11 @@ +================================================================================================ +LongToUnsafeRowMap metrics +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LongToUnsafeRowMap metrics: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +LongToUnsafeRowMap 568 575 5 0.9 1136.4 1.0X + + diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt index 338244ad542f4..7483f517a62e8 100644 --- a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt +++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt @@ -2,10 +2,10 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +LongToUnsafeRowMap metrics: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +LongToUnsafeRowMap 513 629 106 1.0 1026.6 1.0X diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..221ac42022a15 --- /dev/null +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -0,0 +1,25 @@ +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +prepare string w/ interval 672 728 64 1.5 672.1 1.0X +prepare string w/o interval 580 602 19 1.7 580.4 1.2X +1 units w/ interval 9450 9575 138 0.1 9449.6 0.1X +1 units w/o interval 8948 8968 19 0.1 8948.3 0.1X +2 units w/ interval 10947 10966 19 0.1 10947.1 0.1X +2 units w/o interval 10470 10489 26 0.1 10469.5 0.1X +3 units w/ interval 12265 12333 72 0.1 12264.5 0.1X +3 units w/o interval 12001 12004 3 0.1 12000.6 0.1X +4 units w/ interval 13749 13828 69 0.1 13748.5 0.0X +4 units w/o interval 13467 13479 15 0.1 13467.3 0.0X +5 units w/ interval 15392 15446 51 0.1 15392.1 0.0X +5 units w/o interval 15090 15107 29 0.1 15089.7 0.0X +6 units w/ interval 16696 16714 20 0.1 16695.9 0.0X +6 units w/o interval 16361 16366 5 0.1 16361.4 0.0X +7 units w/ interval 18190 18270 71 0.1 18190.2 0.0X +7 units w/o interval 17757 17767 9 0.1 17756.7 0.0X +8 units w/ interval 19821 19870 43 0.1 19820.7 0.0X +8 units w/o interval 19479 19555 97 0.1 19479.5 0.0X +9 units w/ interval 21417 21481 56 0.0 21417.1 0.0X +9 units w/o interval 21058 21131 86 0.0 21058.2 0.0X + diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt new file mode 100644 index 0000000000000..60e8e5198353c --- /dev/null +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -0,0 +1,26 @@ +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +prepare string w/ interval 596 647 61 1.7 596.0 1.0X +prepare string w/o interval 530 554 22 1.9 530.2 1.1X +1 units w/ interval 9168 9243 66 0.1 9167.8 0.1X +1 units w/o interval 8740 8744 5 0.1 8740.2 0.1X +2 units w/ interval 10815 10874 52 0.1 10815.0 0.1X +2 units w/o interval 10413 10419 11 0.1 10412.8 0.1X +3 units w/ interval 12490 12530 37 0.1 12490.3 0.0X +3 units w/o interval 12173 12180 9 0.1 12172.8 0.0X +4 units w/ interval 13788 13834 43 0.1 13788.0 0.0X +4 units w/o interval 13445 13456 10 0.1 13445.5 0.0X +5 units w/ interval 15313 15330 15 0.1 15312.7 0.0X +5 units w/o interval 14928 14942 16 0.1 14928.0 0.0X +6 units w/ interval 16959 17003 42 0.1 16959.1 0.0X +6 units w/o interval 16623 16627 5 0.1 16623.3 0.0X +7 units w/ interval 18955 18972 21 0.1 18955.4 0.0X +7 units w/o interval 18454 18462 7 0.1 18454.1 0.0X +8 units w/ interval 20835 20843 8 0.0 20835.4 0.0X +8 units w/o interval 20446 20463 19 0.0 20445.7 0.0X +9 units w/ interval 22981 23031 43 0.0 22981.4 0.0X +9 units w/o interval 22581 22603 25 0.0 22581.1 0.0X + + diff --git a/sql/core/benchmarks/JSONBenchmark-jdk11-results.txt b/sql/core/benchmarks/JSONBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..56a87d41e3d47 --- /dev/null +++ b/sql/core/benchmarks/JSONBenchmark-jdk11-results.txt @@ -0,0 +1,112 @@ +================================================================================================ +Benchmark for performance of JSON parsing +================================================================================================ + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 70863 71147 417 1.4 708.6 1.0X +UTF-8 is set 133520 133658 131 0.7 1335.2 0.5X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 44462 44655 313 2.2 444.6 1.0X +UTF-8 is set 89342 89386 53 1.1 893.4 0.5X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 58617 59071 541 0.2 5861.7 1.0X +UTF-8 is set 102650 102948 493 0.1 10265.0 0.6X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 168197 168256 61 0.0 336393.0 1.0X +UTF-8 is set 191539 191724 186 0.0 383078.2 0.9X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Select 10 columns 18983 18998 22 0.5 1898.3 1.0X +Select 1 column 23544 23575 32 0.4 2354.4 0.8X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 9336 9402 66 1.1 933.6 1.0X +Short column with UTF-8 13310 13870 489 0.8 1331.0 0.7X +Wide column without encoding 112821 113012 202 0.1 11282.1 0.1X +Wide column with UTF-8 156408 156910 776 0.1 15640.8 0.1X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 743 744 1 13.5 74.3 1.0X +from_json 22379 22566 279 0.4 2237.9 0.0X +json_tuple 32201 32271 89 0.3 3220.1 0.0X +get_json_object 21973 22405 375 0.5 2197.3 0.0X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 3771 3780 8 13.3 75.4 1.0X +schema inferring 34808 35213 506 1.4 696.2 0.1X +parsing 36917 38889 1757 1.4 738.3 0.1X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 11928 11955 38 4.2 238.6 1.0X +Schema inferring 44716 44921 220 1.1 894.3 0.3X +Parsing without charset 35749 35838 77 1.4 715.0 0.3X +Parsing with UTF-8 50521 50530 12 1.0 1010.4 0.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Create a dataset of timestamps 2473 2487 18 4.0 247.3 1.0X +to_json(timestamp) 16100 16284 236 0.6 1610.0 0.2X +write timestamps to files 13226 13852 676 0.8 1322.6 0.2X +Create a dataset of dates 2661 2680 16 3.8 266.1 0.9X +to_json(date) 10696 10730 55 0.9 1069.6 0.2X +write dates to files 8543 8926 384 1.2 854.3 0.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +read timestamp text from files 2822 2829 8 3.5 282.2 1.0X +read timestamps from files 34884 34900 23 0.3 3488.4 0.1X +infer timestamps from files 70310 71551 1079 0.1 7031.0 0.0X +read date text from files 2649 2655 6 3.8 264.9 1.1X +read date from files 22875 22904 33 0.4 2287.5 0.1X +timestamp strings 4025 4059 31 2.5 402.5 0.7X +parse timestamps from Dataset[String] 43040 43157 115 0.2 4304.0 0.1X +infer timestamps from Dataset[String] 81762 81927 144 0.1 8176.2 0.0X +date strings 4899 4913 21 2.0 489.9 0.6X +parse dates from Dataset[String] 33530 33560 27 0.3 3353.0 0.1X +from_json(timestamp) 60762 60877 142 0.2 6076.2 0.0X +from_json(date) 52482 52620 150 0.2 5248.2 0.1X + + diff --git a/sql/core/benchmarks/JSONBenchmark-results.txt b/sql/core/benchmarks/JSONBenchmark-results.txt deleted file mode 100644 index 7846983b44fb3..0000000000000 --- a/sql/core/benchmarks/JSONBenchmark-results.txt +++ /dev/null @@ -1,112 +0,0 @@ -================================================================================================ -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-jdk11-results.txt b/sql/core/benchmarks/JoinBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..06b220586ed94 --- /dev/null +++ b/sql/core/benchmarks/JoinBenchmark-jdk11-results.txt @@ -0,0 +1,75 @@ +================================================================================================ +Join Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w long wholestage off 4872 4888 24 4.3 232.3 1.0X +Join w long wholestage on 337 425 61 62.3 16.1 14.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w long duplicated wholestage off 6290 6346 79 3.3 299.9 1.0X +Join w long duplicated wholestage on 328 347 17 64.0 15.6 19.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w 2 ints wholestage off 174475 174532 82 0.1 8319.6 1.0X +Join w 2 ints wholestage on 165490 167355 1507 0.1 7891.2 1.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w 2 longs wholestage off 7025 7121 135 3.0 335.0 1.0X +Join w 2 longs wholestage on 1878 1954 103 11.2 89.5 3.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w 2 longs duplicated wholestage off 20822 20879 81 1.0 992.9 1.0X +Join w 2 longs duplicated wholestage on 2454 2512 60 8.5 117.0 8.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +outer join w long wholestage off 3900 3907 10 5.4 186.0 1.0X +outer join w long wholestage on 226 235 10 92.6 10.8 17.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +semi join w long wholestage off 2560 2621 87 8.2 122.1 1.0X +semi join w long wholestage on 209 219 15 100.5 10.0 12.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort merge join wholestage off 1707 1712 7 1.2 814.0 1.0X +sort merge join wholestage on 1447 1482 43 1.4 690.0 1.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort merge join with duplicates wholestage off 2293 2307 19 0.9 1093.4 1.0X +sort merge join with duplicates wholestage on 2018 2061 66 1.0 962.1 1.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +shuffle hash join wholestage off 1424 1458 47 2.9 339.6 1.0X +shuffle hash join wholestage on 1203 1227 28 3.5 286.9 1.2X + + diff --git a/sql/core/benchmarks/JoinBenchmark-results.txt b/sql/core/benchmarks/JoinBenchmark-results.txt index 8ceb5e7a7fe94..3b88d5eaccc7e 100644 --- a/sql/core/benchmarks/JoinBenchmark-results.txt +++ b/sql/core/benchmarks/JoinBenchmark-results.txt @@ -2,74 +2,74 @@ Join Benchmark ================================================================================================ -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_222-b10 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 +Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w long wholestage off 4685 4814 182 4.5 223.4 1.0X +Join w long wholestage on 440 524 102 47.7 21.0 10.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_222-b10 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 +Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w long duplicated wholestage off 6266 6291 35 3.3 298.8 1.0X +Join w long duplicated wholestage on 340 359 27 61.7 16.2 18.4X -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_222-b10 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 +Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w 2 ints wholestage off 174733 174916 259 0.1 8331.9 1.0X +Join w 2 ints wholestage on 166815 167619 823 0.1 7954.4 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_222-b10 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 +Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w 2 longs wholestage off 7835 7889 76 2.7 373.6 1.0X +Join w 2 longs wholestage on 2057 2148 73 10.2 98.1 3.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_222-b10 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 +Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Join w 2 longs duplicated wholestage off 19435 19497 87 1.1 926.7 1.0X +Join w 2 longs duplicated wholestage on 2472 2528 57 8.5 117.9 7.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_222-b10 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 +outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +outer join w long wholestage off 3945 3992 66 5.3 188.1 1.0X +outer join w long wholestage on 227 235 11 92.5 10.8 17.4X -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_222-b10 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 +semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +semi join w long wholestage off 2436 2469 46 8.6 116.2 1.0X +semi join w long wholestage on 231 242 11 91.0 11.0 10.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_222-b10 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 +sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort merge join wholestage off 1820 1825 7 1.2 867.9 1.0X +sort merge join wholestage on 1507 1552 34 1.4 718.8 1.2X -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_222-b10 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 +sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort merge join with duplicates wholestage off 2319 2380 86 0.9 1105.7 1.0X +sort merge join with duplicates wholestage on 2087 2139 51 1.0 995.3 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_222-b10 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 +shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +shuffle hash join wholestage off 1554 1591 52 2.7 370.6 1.0X +shuffle hash join wholestage on 1142 1200 51 3.7 272.2 1.4X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt new file mode 100644 index 0000000000000..56d371eade685 --- /dev/null +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -0,0 +1,112 @@ +================================================================================================ +Benchmark for performance of JSON parsing +================================================================================================ + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 69387 69850 407 1.4 693.9 1.0X +UTF-8 is set 112131 112205 83 0.9 1121.3 0.6X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 44542 44671 122 2.2 445.4 1.0X +UTF-8 is set 71793 71945 146 1.4 717.9 0.6X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 58615 59011 347 0.2 5861.5 1.0X +UTF-8 is set 102542 102719 153 0.1 10254.2 0.6X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 168861 170014 1552 0.0 337722.0 1.0X +UTF-8 is set 191140 191250 112 0.0 382280.3 0.9X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Select 10 columns 28017 28066 47 0.4 2801.7 1.0X +Select 1 column 24590 24641 67 0.4 2459.0 1.1X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 17179 17465 487 0.6 1717.9 1.0X +Short column with UTF-8 25173 25255 139 0.4 2517.3 0.7X +Wide column without encoding 146956 147069 104 0.1 14695.6 0.1X +Wide column with UTF-8 196626 197233 549 0.1 19662.6 0.1X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 1206 1212 5 8.3 120.6 1.0X +from_json 27641 27680 34 0.4 2764.1 0.0X +json_tuple 43404 44377 860 0.2 4340.4 0.0X +get_json_object 26821 27239 619 0.4 2682.1 0.0X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 5842 5865 33 8.6 116.8 1.0X +schema inferring 69673 70082 478 0.7 1393.5 0.1X +parsing 78805 81812 NaN 0.6 1576.1 0.1X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 10782 10801 18 4.6 215.6 1.0X +Schema inferring 76817 77205 623 0.7 1536.3 0.1X +Parsing without charset 90638 91395 794 0.6 1812.8 0.1X +Parsing with UTF-8 120085 121975 1705 0.4 2401.7 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Create a dataset of timestamps 4706 4717 9 2.1 470.6 1.0X +to_json(timestamp) 29447 29615 226 0.3 2944.7 0.2X +write timestamps to files 20251 20673 503 0.5 2025.1 0.2X +Create a dataset of dates 4157 4172 18 2.4 415.7 1.1X +to_json(date) 21267 21301 53 0.5 2126.7 0.2X +write dates to files 13477 13897 485 0.7 1347.7 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +read timestamp text from files 2666 2687 29 3.8 266.6 1.0X +read timestamps from files 46967 47354 438 0.2 4696.7 0.1X +infer timestamps from files 97693 97745 65 0.1 9769.3 0.0X +read date text from files 2594 2599 5 3.9 259.4 1.0X +read date from files 35796 36008 195 0.3 3579.6 0.1X +timestamp strings 6367 6424 84 1.6 636.7 0.4X +parse timestamps from Dataset[String] 58863 59255 340 0.2 5886.3 0.0X +infer timestamps from Dataset[String] 114148 114820 836 0.1 11414.8 0.0X +date strings 7847 7863 22 1.3 784.7 0.3X +parse dates from Dataset[String] 49085 49289 212 0.2 4908.5 0.1X +from_json(timestamp) 77030 77335 395 0.1 7703.0 0.0X +from_json(date) 63275 63290 15 0.2 6327.5 0.0X + + diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..f7332d4cea16e --- /dev/null +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt @@ -0,0 +1,22 @@ +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +prepare make_date() 2952 3104 224 33.9 29.5 1.0X +make_date(2019, 9, 16) 2341 2585 359 42.7 23.4 1.3X +make_date(*, *, *) 4751 4808 53 21.0 47.5 0.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +prepare make_timestamp() 3462 3585 143 0.3 3461.8 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 114 120 6 8.7 114.4 30.3X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 74 82 13 13.5 74.0 46.8X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 82 95 11 12.1 82.5 42.0X +make_timestamp(*, *, *, 3, 4, 50.123456) 293 308 14 3.4 292.9 11.8X +make_timestamp(*, *, *, *, *, 0) 278 284 9 3.6 278.1 12.4X +make_timestamp(*, *, *, *, *, 60.0) 287 289 2 3.5 287.3 12.0X +make_timestamp(2019, 1, 2, *, *, *) 3556 3574 21 0.3 3555.7 1.0X +make_timestamp(*, *, *, *, *, *) 3578 3590 16 0.3 3577.9 1.0X + diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt index a3c89d643e912..2b184fd26eeeb 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -1,22 +1,22 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2149 2289 196 46.5 21.5 1.0X -make_date(2019, 9, 16) 1829 1868 58 54.7 18.3 1.2X -make_date(*, *, *) 3180 3339 139 31.4 31.8 0.7X +prepare make_date() 2990 3159 267 33.4 29.9 1.0X +make_date(2019, 9, 16) 2376 2446 90 42.1 23.8 1.3X +make_date(*, *, *) 4751 4786 31 21.0 47.5 0.6X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_timestamp() 2950 3025 96 0.3 2950.3 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 45 47 1 22.1 45.2 65.3X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 42 42 1 24.0 41.7 70.8X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 41 42 1 24.2 41.3 71.4X -make_timestamp(*, *, *, 3, 4, 50.123456) 252 256 7 4.0 251.5 11.7X -make_timestamp(*, *, *, *, *, 0) 225 227 3 4.5 224.6 13.1X -make_timestamp(*, *, *, *, *, 60.0) 230 233 2 4.3 230.4 12.8X -make_timestamp(2019, 1, 2, *, *, *) 3078 3118 35 0.3 3078.5 1.0X -make_timestamp(*, *, *, *, *, *) 3092 3109 17 0.3 3092.4 1.0X +prepare make_timestamp() 3855 3936 104 0.3 3855.3 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 83 87 5 12.0 83.1 46.4X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 73 82 9 13.8 72.7 53.0X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 66 74 9 15.2 65.8 58.6X +make_timestamp(*, *, *, 3, 4, 50.123456) 266 277 11 3.8 265.9 14.5X +make_timestamp(*, *, *, *, *, 0) 268 275 13 3.7 267.6 14.4X +make_timestamp(*, *, *, *, *, 60.0) 272 273 0 3.7 272.4 14.2X +make_timestamp(2019, 1, 2, *, *, *) 3940 3966 24 0.3 3940.4 1.0X +make_timestamp(*, *, *, *, *, *) 3867 3917 68 0.3 3867.4 1.0X diff --git a/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt b/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..991a447fe3237 --- /dev/null +++ b/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt @@ -0,0 +1,127 @@ +================================================================================================ +filter & aggregate without group +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +range/filter/sum wholestage off 52841 53513 951 39.7 25.2 1.0X +range/filter/sum wholestage on 3104 3188 109 675.6 1.5 17.0X + + +================================================================================================ +range/limit/sum +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +range/limit/sum wholestage off 199 222 32 2633.5 0.4 1.0X +range/limit/sum wholestage on 131 138 7 3991.0 0.3 1.5X + + +================================================================================================ +sample +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sample with replacement wholestage off 13015 13441 602 10.1 99.3 1.0X +sample with replacement wholestage on 7688 7745 91 17.0 58.7 1.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sample without replacement wholestage off 2990 2993 4 43.8 22.8 1.0X +sample without replacement wholestage on 1172 1186 13 111.9 8.9 2.6X + + +================================================================================================ +collect +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +collect 1 million 317 425 188 3.3 301.9 1.0X +collect 2 millions 575 598 26 1.8 548.2 0.6X +collect 4 millions 1350 1848 704 0.8 1287.6 0.2X + + +================================================================================================ +collect limit +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +collect limit 1 million 375 387 14 2.8 358.0 1.0X +collect limit 2 millions 714 736 28 1.5 681.3 0.5X + + +================================================================================================ +generate explode +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate explode array wholestage off 19669 19829 227 0.9 1172.3 1.0X +generate explode array wholestage on 10983 11020 35 1.5 654.7 1.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate explode map wholestage off 57756 57928 243 0.3 3442.5 1.0X +generate explode map wholestage on 47398 47703 250 0.4 2825.2 1.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate posexplode array wholestage off 21402 21525 174 0.8 1275.7 1.0X +generate posexplode array wholestage on 11898 11982 67 1.4 709.2 1.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate inline array wholestage off 15570 15597 37 1.1 928.1 1.0X +generate inline array wholestage on 10044 10161 87 1.7 598.6 1.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate big struct array wholestage off 601 615 19 0.1 10023.5 1.0X +generate big struct array wholestage on 388 410 25 0.2 6458.7 1.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 492 525 48 0.1 8191.7 1.0X +generate big nested struct array wholestage on 468 496 30 0.1 7796.4 1.1X + + +================================================================================================ +generate regular generator +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate stack wholestage off 28014 28091 109 0.6 1669.8 1.0X +generate stack wholestage on 19246 19267 27 0.9 1147.2 1.5X + + diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt index c4ae052095656..012e81b03c8df 100644 --- a/sql/core/benchmarks/MiscBenchmark-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-results.txt @@ -2,126 +2,126 @@ filter & aggregate without group ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -range/filter/sum wholestage off 47752 / 48952 43.9 22.8 1.0X -range/filter/sum wholestage on 3123 / 3558 671.5 1.5 15.3X +range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +range/filter/sum wholestage off 45894 47528 2310 45.7 21.9 1.0X +range/filter/sum wholestage on 3193 3572 213 656.8 1.5 14.4X ================================================================================================ range/limit/sum ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -range/limit/sum wholestage off 229 / 236 2288.9 0.4 1.0X -range/limit/sum wholestage on 257 / 267 2041.0 0.5 0.9X +range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +range/limit/sum wholestage off 202 207 7 2593.1 0.4 1.0X +range/limit/sum wholestage on 148 161 20 3545.1 0.3 1.4X ================================================================================================ sample ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sample with replacement wholestage off 12908 / 13076 10.2 98.5 1.0X -sample with replacement wholestage on 7334 / 7346 17.9 56.0 1.8X +sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sample with replacement wholestage off 12337 12657 452 10.6 94.1 1.0X +sample with replacement wholestage on 7355 7368 14 17.8 56.1 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sample without replacement wholestage off 3082 / 3095 42.5 23.5 1.0X -sample without replacement wholestage on 1125 / 1211 116.5 8.6 2.7X +sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sample without replacement wholestage off 3050 3060 14 43.0 23.3 1.0X +sample without replacement wholestage on 1103 1118 22 118.8 8.4 2.8X ================================================================================================ collect ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -collect 1 million 291 / 311 3.6 277.3 1.0X -collect 2 millions 552 / 564 1.9 526.6 0.5X -collect 4 millions 1104 / 1108 0.9 1053.0 0.3X +collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +collect 1 million 324 361 41 3.2 309.4 1.0X +collect 2 millions 619 700 70 1.7 590.4 0.5X +collect 4 millions 1214 1293 111 0.9 1158.2 0.3X ================================================================================================ collect limit ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -collect limit 1 million 311 / 340 3.4 296.2 1.0X -collect limit 2 millions 581 / 614 1.8 554.4 0.5X +collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +collect limit 1 million 391 396 5 2.7 373.0 1.0X +collect limit 2 millions 746 769 26 1.4 711.8 0.5X ================================================================================================ generate explode ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate explode array wholestage off 15211 / 15368 1.1 906.6 1.0X -generate explode array wholestage on 10761 / 10776 1.6 641.4 1.4X +generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate explode array wholestage off 15063 16007 1335 1.1 897.8 1.0X +generate explode array wholestage on 10909 10932 20 1.5 650.2 1.4X -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate explode map wholestage off 22128 / 22578 0.8 1318.9 1.0X -generate explode map wholestage on 16421 / 16520 1.0 978.8 1.3X +generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate explode map wholestage off 55011 55580 806 0.3 3278.9 1.0X +generate explode map wholestage on 46009 46445 399 0.4 2742.3 1.2X -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate posexplode array wholestage off 17108 / 18019 1.0 1019.7 1.0X -generate posexplode array wholestage on 11715 / 11804 1.4 698.3 1.5X +generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate posexplode array wholestage off 17987 18866 1244 0.9 1072.1 1.0X +generate posexplode array wholestage on 11400 11444 38 1.5 679.5 1.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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate inline array wholestage off 16358 / 16418 1.0 975.0 1.0X -generate inline array wholestage on 11152 / 11472 1.5 664.7 1.5X +generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate inline array wholestage off 14905 14991 122 1.1 888.4 1.0X +generate inline array wholestage on 9919 10129 160 1.7 591.2 1.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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -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 +generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate big struct array wholestage off 508 577 98 0.1 8474.0 1.0X +generate big struct array wholestage on 451 485 26 0.1 7523.2 1.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 big nested struct array wholestage off 537 567 43 0.1 8944.4 1.0X +generate big nested struct array wholestage on 528 546 24 0.1 8804.2 1.0X ================================================================================================ generate regular generator ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate stack wholestage off 29082 / 29393 0.6 1733.4 1.0X -generate stack wholestage on 21066 / 21128 0.8 1255.6 1.4X +generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate stack wholestage off 27220 27608 548 0.6 1622.5 1.0X +generate stack wholestage on 20070 20170 117 0.8 1196.3 1.4X diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..4d35f872871f0 --- /dev/null +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt @@ -0,0 +1,53 @@ +================================================================================================ +Nested Schema Pruning Benchmark For ORC v1 +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 150 192 23 6.7 150.1 1.0X +Nested column 1241 1289 82 0.8 1241.3 0.1X +Nested column in array 5466 5574 114 0.2 5465.6 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 463 515 30 2.2 462.6 1.0X +Nested column 1920 1967 27 0.5 1920.1 0.2X +Nested column in array 6565 6685 123 0.2 6564.8 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 387 431 80 2.6 387.0 1.0X +Nested column 1846 1865 13 0.5 1846.2 0.2X +Nested column in array 6458 6569 114 0.2 6458.2 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 391 405 13 2.6 390.9 1.0X +Nested column 4597 4684 81 0.2 4597.4 0.1X +Nested column in array 9434 9499 54 0.1 9433.9 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 131 165 27 7.6 130.9 1.0X +Nested column 1258 1298 31 0.8 1257.7 0.1X +Nested column in array 5360 5393 16 0.2 5359.7 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 584 632 63 1.7 583.7 1.0X +Nested column 5211 5384 108 0.2 5211.1 0.1X +Nested column in array 10279 10397 94 0.1 10279.0 0.1X + + diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt index 765193d6c6436..0dd85968b937e 100644 --- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 131 167 24 7.6 131.0 1.0X +Nested column 1296 1340 33 0.8 1296.0 0.1X +Nested column in array 5568 5745 243 0.2 5567.8 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 432 468 33 2.3 431.6 1.0X +Nested column 1778 1828 35 0.6 1777.7 0.2X +Nested column in array 6565 6727 137 0.2 6565.1 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 365 406 60 2.7 365.3 1.0X +Nested column 1803 1821 22 0.6 1803.2 0.2X +Nested column in array 6453 6530 111 0.2 6453.5 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 370 390 30 2.7 369.6 1.0X +Nested column 4955 5028 61 0.2 4955.0 0.1X +Nested column in array 10490 10553 49 0.1 10490.5 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 135 150 18 7.4 134.5 1.0X +Nested column 1615 1717 78 0.6 1615.0 0.1X +Nested column in array 6919 7024 53 0.1 6919.4 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 580 619 28 1.7 580.4 1.0X +Nested column 5547 5752 185 0.2 5547.3 0.1X +Nested column in array 11639 11838 183 0.1 11639.4 0.0X diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0036510d62fc9 --- /dev/null +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt @@ -0,0 +1,53 @@ +================================================================================================ +Nested Schema Pruning Benchmark For ORC v2 +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 150 187 20 6.7 150.2 1.0X +Nested column 1370 1463 104 0.7 1369.8 0.1X +Nested column in array 6575 6656 66 0.2 6575.2 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 134 161 20 7.5 133.7 1.0X +Nested column 1532 1550 15 0.7 1532.4 0.1X +Nested column in array 6601 6688 126 0.2 6601.4 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 409 491 190 2.4 409.5 1.0X +Nested column 1896 1938 60 0.5 1896.2 0.2X +Nested column in array 7414 7472 43 0.1 7414.0 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 408 433 37 2.5 408.1 1.0X +Nested column 4703 4795 101 0.2 4703.4 0.1X +Nested column in array 10477 10556 61 0.1 10476.6 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 164 31 7.7 130.3 1.0X +Nested column 1471 1513 44 0.7 1470.5 0.1X +Nested column in array 6492 6551 55 0.2 6492.0 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 268 283 14 3.7 268.4 1.0X +Nested column 3347 3401 77 0.3 3347.3 0.1X +Nested column in array 9297 9433 237 0.1 9296.6 0.0X + + diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt index fdd347f4bad9b..8aff152f0c136 100644 --- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 141 161 22 7.1 140.6 1.0X +Nested column 1425 1455 26 0.7 1424.7 0.1X +Nested column in array 5248 5300 46 0.2 5247.5 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 133 163 22 7.5 132.8 1.0X +Nested column 1254 1308 40 0.8 1254.0 0.1X +Nested column in array 5303 5418 81 0.2 5303.3 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 377 401 19 2.7 376.7 1.0X +Nested column 1676 1722 21 0.6 1676.1 0.2X +Nested column in array 6019 6127 109 0.2 6018.7 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 390 447 151 2.6 390.1 1.0X +Nested column 4300 4364 60 0.2 4299.7 0.1X +Nested column in array 8832 9030 114 0.1 8832.4 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 132 143 7 7.6 131.6 1.0X +Nested column 1260 1303 20 0.8 1260.2 0.1X +Nested column in array 5359 5453 74 0.2 5359.1 0.0X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 288 302 20 3.5 287.6 1.0X +Nested column 3169 3242 53 0.3 3168.7 0.1X +Nested column in array 8151 8301 123 0.1 8151.3 0.0X diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..4535220aeab0b --- /dev/null +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt @@ -0,0 +1,53 @@ +================================================================================================ +Nested Schema Pruning Benchmark For Parquet +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 164 191 18 6.1 163.6 1.0X +Nested column 416 444 26 2.4 415.7 0.4X +Nested column in array 1349 1398 39 0.7 1349.4 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 153 169 14 6.5 152.7 1.0X +Nested column 540 566 26 1.9 540.5 0.3X +Nested column in array 1378 1447 87 0.7 1378.0 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 414 433 26 2.4 414.0 1.0X +Nested column 736 777 56 1.4 736.2 0.6X +Nested column in array 1895 1954 86 0.5 1895.2 0.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 402 436 49 2.5 402.1 1.0X +Nested column 3190 3225 26 0.3 3190.4 0.1X +Nested column in array 3749 3806 68 0.3 3748.7 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 135 159 25 7.4 134.5 1.0X +Nested column 462 497 36 2.2 462.4 0.3X +Nested column in array 1453 1508 45 0.7 1453.5 0.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 277 301 27 3.6 277.3 1.0X +Nested column 2057 2097 33 0.5 2057.3 0.1X +Nested column in array 3027 3134 122 0.3 3027.4 0.1X + + diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt index 4e0c368b5370e..182a1aed497a3 100644 --- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 149 183 22 6.7 148.8 1.0X +Nested column 413 436 20 2.4 413.4 0.4X +Nested column in array 1309 1327 24 0.8 1308.9 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 160 184 17 6.3 159.5 1.0X +Nested column 436 481 77 2.3 436.5 0.4X +Nested column in array 1354 1381 26 0.7 1353.6 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 378 412 61 2.6 378.0 1.0X +Nested column 744 790 51 1.3 744.0 0.5X +Nested column in array 1915 1987 76 0.5 1915.0 0.2X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 389 407 16 2.6 388.6 1.0X +Nested column 3152 3217 48 0.3 3151.9 0.1X +Nested column in array 3698 3860 274 0.3 3698.2 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 143 170 27 7.0 142.7 1.0X +Nested column 450 461 8 2.2 449.9 0.3X +Nested column in array 1411 1460 41 0.7 1411.2 0.1X -OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +Top-level column 293 309 22 3.4 292.9 1.0X +Nested column 2109 2142 19 0.5 2109.1 0.1X +Nested column in array 3018 3096 79 0.3 3017.7 0.1X diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk11-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0e5a051d92c90 --- /dev/null +++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk11-results.txt @@ -0,0 +1,12 @@ +================================================================================================ +Write primitive arrays in dataset +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 636 681 50 13.2 75.8 1.0X +Double 727 809 71 11.5 86.7 0.9X + + diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt index 5fa461fecd4d2..837145e7c93a5 100644 --- a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt +++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt @@ -6,7 +6,7 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 543 605 67 15.4 64.7 1.0X -Double 737 776 36 11.4 87.9 0.7X +Int 631 684 65 13.3 75.2 1.0X +Double 792 854 61 10.6 94.4 0.8X diff --git a/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt b/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..dc86fb686fb77 --- /dev/null +++ b/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt @@ -0,0 +1,15 @@ +================================================================================================ +range +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +full scan 20357 21595 1692 25.8 38.8 1.0X +limit after range 115 130 15 4563.9 0.2 177.2X +filter after range 1890 1906 21 277.5 3.6 10.8X +count after range 85 87 3 6200.7 0.2 240.8X +count after limit after range 101 109 9 5213.9 0.2 202.4X + + diff --git a/sql/core/benchmarks/RangeBenchmark-results.txt b/sql/core/benchmarks/RangeBenchmark-results.txt index 21766e0fd8664..7b6daf9b9c40d 100644 --- a/sql/core/benchmarks/RangeBenchmark-results.txt +++ b/sql/core/benchmarks/RangeBenchmark-results.txt @@ -2,15 +2,14 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +full scan 19094 19768 459 27.5 36.4 1.0X +limit after range 90 108 19 5803.9 0.2 211.4X +filter after range 1807 1820 12 290.1 3.4 10.6X +count after range 88 93 6 5941.4 0.2 216.4X +count after limit after range 82 86 4 6398.2 0.2 233.0X diff --git a/sql/core/benchmarks/SortBenchmark-jdk11-results.txt b/sql/core/benchmarks/SortBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..a61989b174bf5 --- /dev/null +++ b/sql/core/benchmarks/SortBenchmark-jdk11-results.txt @@ -0,0 +1,16 @@ +================================================================================================ +radix sort +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +radix sort 25000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +reference TimSort key prefix array 15588 15592 6 1.6 623.5 1.0X +reference Arrays.sort 3367 3377 14 7.4 134.7 4.6X +radix sort one byte 436 447 10 57.3 17.5 35.7X +radix sort two bytes 875 885 9 28.6 35.0 17.8X +radix sort eight bytes 3409 3429 27 7.3 136.4 4.6X +radix sort key prefix array 5985 6058 104 4.2 239.4 2.6X + + diff --git a/sql/core/benchmarks/SortBenchmark-results.txt b/sql/core/benchmarks/SortBenchmark-results.txt index 0d00a0c89d02d..b321d1115b727 100644 --- a/sql/core/benchmarks/SortBenchmark-results.txt +++ b/sql/core/benchmarks/SortBenchmark-results.txt @@ -2,16 +2,15 @@ radix sort ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_162-b12 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - -radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -reference TimSort key prefix array 11770 / 11960 2.1 470.8 1.0X -reference Arrays.sort 2106 / 2128 11.9 84.3 5.6X -radix sort one byte 93 / 100 269.7 3.7 126.9X -radix sort two bytes 171 / 179 146.0 6.9 68.7X -radix sort eight bytes 659 / 664 37.9 26.4 17.9X -radix sort key prefix array 1024 / 1053 24.4 41.0 11.5X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +radix sort 25000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +reference TimSort key prefix array 15736 15778 59 1.6 629.4 1.0X +reference Arrays.sort 3051 3057 10 8.2 122.0 5.2X +radix sort one byte 442 453 10 56.6 17.7 35.6X +radix sort two bytes 883 885 1 28.3 35.3 17.8X +radix sort eight bytes 3422 3440 26 7.3 136.9 4.6X +radix sort key prefix array 6025 6138 159 4.1 241.0 2.6X diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt new file mode 100644 index 0000000000000..87cf48c5a11f2 --- /dev/null +++ b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt @@ -0,0 +1,810 @@ +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q1 1661 1738 109 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q2 2517 2601 119 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q3 544 566 36 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q4 15391 15676 403 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q5 3347 3709 512 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q6 2461 2469 11 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q7 1195 1216 30 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q8 1164 1207 60 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q9 2893 2932 54 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q10 3685 3840 219 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q11 3791 3820 41 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q12 553 575 18 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q13 2960 3157 278 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14a 26682 27183 709 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14b 20780 21353 811 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q15 867 883 20 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q16 1448 1484 52 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q17 3104 3147 60 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q18 2299 2555 362 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q19 881 897 17 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q20 539 583 41 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q21 1257 1268 15 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q22 4142 4340 281 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q23a 15861 16068 292 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q23b 19762 19835 103 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q24a 3686 3808 173 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q24b 3364 3396 46 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q25 3004 3294 410 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q26 886 917 28 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q27 1382 1382 0 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q28 3329 3354 35 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q29 2949 3048 141 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q30 1443 1443 0 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q31 2658 2859 285 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q32 707 733 23 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q33 1790 1798 12 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q34 880 886 6 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q35 3152 3277 177 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q36 1211 1229 26 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q37 1495 1500 7 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q38 2017 2073 79 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q39a 2575 2603 39 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q39b 2388 2507 168 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q40 872 920 81 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q41 453 518 48 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q42 442 480 30 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q43 683 691 7 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q44 1301 1307 8 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q45 700 717 26 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q46 1198 1224 38 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q47 4915 5163 350 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q48 2569 2585 22 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q49 2193 2277 119 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q50 1558 1564 9 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q51 4375 4515 198 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q52 441 454 15 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q53 719 751 29 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q54 3141 3182 58 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q55 433 471 24 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q56 1766 1811 65 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q57 3203 3378 248 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q58 2170 2236 94 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q59 2318 2322 6 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q60 1806 1842 51 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q61 1635 1643 10 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q62 629 648 29 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q63 708 730 21 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q64 8943 9106 231 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q65 1384 1432 68 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q66 2225 2334 154 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q67 9937 9994 80 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q68 1152 1155 4 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q69 3333 3393 86 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q70 1472 1498 36 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q71 1301 1328 38 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q72 131085 131881 1126 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q73 844 859 26 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q74 3230 3269 55 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q75 5169 5503 472 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q76 1149 1170 30 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q77 2225 2534 437 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q78 5088 5280 272 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q79 1065 1069 6 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q80 3555 3754 282 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q81 1249 1253 6 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q82 2167 2186 26 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q83 1379 1445 94 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q84 1375 1397 31 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q85 4318 4621 427 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q86 715 726 16 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q87 2486 2520 49 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q88 3302 3430 182 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q89 854 914 93 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q90 622 635 10 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q91 1033 1052 26 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q92 649 684 31 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q93 1331 1366 49 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q94 1033 1042 13 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q95 5701 5709 12 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q96 438 462 20 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q97 1882 1956 104 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q98 728 765 46 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q99 728 736 8 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q5a-v2.7 5933 6010 108 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q6-v2.7 2322 2379 80 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q10a-v2.7 3946 3947 2 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q11-v2.7 3784 3837 75 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q12-v2.7 475 541 75 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14-v2.7 19842 20008 235 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14a-v2.7 127135 127517 539 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q18a-v2.7 5413 6082 946 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q20-v2.7 535 552 18 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q22-v2.7 20164 20228 91 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q22a-v2.7 9719 9961 342 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q24-v2.7 3210 3278 97 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q27a-v2.7 2729 2862 188 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q34-v2.7 877 901 30 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q35-v2.7 2976 3126 212 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q35a-v2.7 3449 3544 134 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q36a-v2.7 2814 2850 51 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q47-v2.7 4699 5077 534 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q49-v2.7 2202 2330 182 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q51a-v2.7 28603 28823 311 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q57-v2.7 3007 3239 329 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q64-v2.7 8686 9350 940 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q67a-v2.7 18643 18876 330 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q70a-v2.7 2990 3028 54 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q72-v2.7 127577 127803 319 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q74-v2.7 3086 3144 82 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q75-v2.7 5158 5431 386 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q77a-v2.7 5109 5406 420 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q78-v2.7 4743 5234 694 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q80a-v2.7 6636 7007 525 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q86a-v2.7 1651 1661 14 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q98-v2.7 637 699 93 0.0 Infinity 1.0X + diff --git a/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt b/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..3454c1bccda20 --- /dev/null +++ b/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt @@ -0,0 +1,59 @@ +================================================================================================ +UDF with mixed input types +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 289 302 18 0.3 2893.0 1.0X +long/nullable int/string to string wholestage on 141 165 20 0.7 1405.4 2.1X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 112 120 11 0.9 1124.6 1.0X +long/nullable int/string to option wholestage on 68 77 8 1.5 681.3 1.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 74 79 6 1.3 744.3 1.0X +long/nullable int/string to primitive wholestage on 71 83 10 1.4 713.5 1.0X + + +================================================================================================ +UDF with primitive types +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 94 99 6 1.1 942.4 1.0X +long/nullable int to string wholestage on 58 60 1 1.7 584.3 1.6X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 43 46 5 2.3 425.7 1.0X +long/nullable int to option wholestage on 42 49 7 2.4 423.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +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 40 41 1 2.5 401.7 1.0X +long/nullable int to primitive wholestage on 40 44 5 2.5 400.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Baseline 35 40 6 2.9 348.2 1.0X +With identity UDF 38 38 0 2.6 380.4 0.9X + + diff --git a/sql/core/benchmarks/UDFBenchmark-results.txt b/sql/core/benchmarks/UDFBenchmark-results.txt index 3dfd0c1caeb28..1490b4439640c 100644 --- a/sql/core/benchmarks/UDFBenchmark-results.txt +++ b/sql/core/benchmarks/UDFBenchmark-results.txt @@ -2,58 +2,58 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +long/nullable int/string to string wholestage off 242 308 94 0.4 2416.5 1.0X +long/nullable int/string to string wholestage on 141 179 25 0.7 1410.3 1.7X -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +long/nullable int/string to option wholestage off 91 97 9 1.1 908.1 1.0X +long/nullable int/string to option wholestage on 77 91 12 1.3 774.4 1.2X -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +long/nullable int/string to primitive wholestage off 68 75 9 1.5 684.7 1.0X +long/nullable int/string to primitive wholestage on 62 64 3 1.6 619.8 1.1X ================================================================================================ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +long/nullable int to string wholestage off 67 68 0 1.5 672.2 1.0X +long/nullable int to string wholestage on 66 72 11 1.5 660.5 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +long/nullable int to option wholestage off 53 57 5 1.9 528.1 1.0X +long/nullable int to option wholestage on 41 44 4 2.4 410.2 1.3X -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +long/nullable int to primitive wholestage off 46 48 4 2.2 459.1 1.0X +long/nullable int to primitive wholestage on 40 40 0 2.5 398.0 1.2X -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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 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 +Baseline 53 58 4 1.9 525.6 1.0X +With identity UDF 38 38 0 2.7 376.3 1.4X diff --git a/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk11-results.txt b/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..577cdb0db62b3 --- /dev/null +++ b/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk11-results.txt @@ -0,0 +1,33 @@ +================================================================================================ +Benchmark UnsafeArrayData +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Read UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 183 184 0 914.7 1.1 1.0X +Double 188 189 0 891.1 1.1 1.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 31 32 1 670.0 1.5 1.0X +Double 70 71 1 300.3 3.3 0.4X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Get primitive array from UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 71 73 3 887.3 1.1 1.0X +Double 144 151 6 438.2 2.3 0.5X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Create UnsafeArrayData from primitive array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 69 70 1 906.7 1.1 1.0X +Double 140 142 1 448.8 2.2 0.5X + + diff --git a/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt b/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt index 4ecc1f1fad4b9..6c4669d7ea442 100644 --- a/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt +++ b/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt @@ -2,32 +2,32 @@ Benchmark UnsafeArrayData ================================================================================================ -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Int 233 / 234 718.6 1.4 1.0X -Double 244 / 244 687.0 1.5 1.0X +Read UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 183 183 0 919.0 1.1 1.0X +Double 192 196 4 872.4 1.1 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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Int 32 / 33 658.6 1.5 1.0X -Double 73 / 75 287.0 3.5 0.4X +Write UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 33 36 3 639.3 1.6 1.0X +Double 76 79 2 274.6 3.6 0.4X -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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Int 70 / 72 895.0 1.1 1.0X -Double 141 / 143 446.9 2.2 0.5X +Get primitive array from UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 71 76 4 890.8 1.1 1.0X +Double 147 151 6 427.4 2.3 0.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_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Int 72 / 73 874.7 1.1 1.0X -Double 145 / 146 433.7 2.3 0.5X +Create UnsafeArrayData from primitive array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 73 75 3 866.8 1.2 1.0X +Double 148 150 1 424.8 2.4 0.5X diff --git a/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt b/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..5ba8e8ff9a62b --- /dev/null +++ b/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt @@ -0,0 +1,17 @@ +================================================================================================ +projection on wide table +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS 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 Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +split threshold 10 42942 43023 108 0.0 40952.8 1.0X +split threshold 100 37236 37310 93 0.0 35510.9 1.2X +split threshold 1024 12198 12260 56 0.1 11633.1 3.5X +split threshold 2048 9789 9860 76 0.1 9335.2 4.4X +split threshold 4096 10120 10161 63 0.1 9651.0 4.2X +split threshold 8192 9961 9972 7 0.1 9499.1 4.3X +split threshold 65536 58987 59087 95 0.0 56254.5 0.7X + + diff --git a/sql/core/benchmarks/WideTableBenchmark-results.txt b/sql/core/benchmarks/WideTableBenchmark-results.txt index 8c09f9ca11307..eeb8c34232382 100644 --- a/sql/core/benchmarks/WideTableBenchmark-results.txt +++ b/sql/core/benchmarks/WideTableBenchmark-results.txt @@ -2,16 +2,16 @@ 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 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 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 +projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +split threshold 10 43737 44157 345 0.0 41711.3 1.0X +split threshold 100 33514 35770 1455 0.0 31961.3 1.3X +split threshold 1024 14959 15206 256 0.1 14265.8 2.9X +split threshold 2048 12024 12193 171 0.1 11467.2 3.6X +split threshold 4096 11378 11869 450 0.1 10851.1 3.8X +split threshold 8192 10588 10797 154 0.1 10097.8 4.1X +split threshold 65536 56801 57025 130 0.0 54169.4 0.8X diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02ed6f8adaa62..3a8b7d22397ff 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -38,7 +38,6 @@ com.univocity univocity-parsers - 2.7.3 jar @@ -215,4 +214,16 @@
+ + + + scala-2.13 + + + org.scala-lang.modules + scala-parallel-collections_${scala.binary.version} + + + + diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 863d80b5cb9c5..90b55a8586de7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.function.Supplier; -import scala.collection.AbstractIterator; import scala.collection.Iterator; import scala.math.Ordering; @@ -52,6 +51,12 @@ public final class UnsafeExternalRowSorter { private final UnsafeExternalRowSorter.PrefixComputer prefixComputer; private final UnsafeExternalSorter sorter; + // This flag makes sure the cleanupResource() has been called. After the cleanup work, + // iterator.next should always return false. Downstream operator triggers the resource + // cleanup while they found there's no need to keep the iterator any more. + // See more details in SPARK-21492. + private boolean isReleased = false; + public abstract static class PrefixComputer { public static class Prefix { @@ -157,11 +162,12 @@ public long getSortTimeNanos() { return sorter.getSortTimeNanos(); } - private void cleanupResources() { + public void cleanupResources() { + isReleased = true; sorter.cleanupResources(); } - public Iterator sort() throws IOException { + public Iterator sort() throws IOException { try { final UnsafeSorterIterator sortedIterator = sorter.getSortedIterator(); if (!sortedIterator.hasNext()) { @@ -169,31 +175,32 @@ public Iterator sort() throws IOException { // here in order to prevent memory leaks. cleanupResources(); } - return new AbstractIterator() { + return new RowIterator() { private final int numFields = schema.length(); private UnsafeRow row = new UnsafeRow(numFields); @Override - public boolean hasNext() { - return sortedIterator.hasNext(); - } - - @Override - public UnsafeRow next() { + public boolean advanceNext() { try { - sortedIterator.loadNext(); - row.pointTo( - sortedIterator.getBaseObject(), - sortedIterator.getBaseOffset(), - sortedIterator.getRecordLength()); - if (!hasNext()) { - UnsafeRow copy = row.copy(); // so that we don't have dangling pointers to freed page - row = null; // so that we don't keep references to the base object - cleanupResources(); - return copy; + if (!isReleased && sortedIterator.hasNext()) { + sortedIterator.loadNext(); + row.pointTo( + sortedIterator.getBaseObject(), + sortedIterator.getBaseOffset(), + sortedIterator.getRecordLength()); + // Here is the initial bug fix in SPARK-9364: the bug fix of use-after-free bug + // when returning the last row from an iterator. For example, in + // [[GroupedIterator]], we still use the last row after traversing the iterator + // in `fetchNextGroupIterator` + if (!sortedIterator.hasNext()) { + row = row.copy(); // so that we don't have dangling pointers to freed page + cleanupResources(); + } + return true; } else { - return row; + row = null; // so that we don't keep references to the base object + return false; } } catch (IOException e) { cleanupResources(); @@ -203,14 +210,18 @@ public UnsafeRow next() { } throw new RuntimeException("Exception should have been re-thrown in next()"); } - }; + + @Override + public UnsafeRow getRow() { return row; } + + }.toScala(); } catch (IOException e) { cleanupResources(); throw e; } } - public Iterator sort(Iterator inputIterator) throws IOException { + public Iterator sort(Iterator inputIterator) throws IOException { while (inputIterator.hasNext()) { insertRow(inputIterator.next()); } 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 7b903a3f7f148..ed10843b08596 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 @@ -200,7 +200,7 @@ class Column(val expr: Expression) extends Logging { UnresolvedAlias(a, Some(Column.generateAlias)) // Wait until the struct is resolved. This will generate a nicer looking alias. - case struct: CreateNamedStructLike => UnresolvedAlias(struct) + case struct: CreateNamedStruct => UnresolvedAlias(struct) case expr: Expression => Alias(expr, toPrettySQL(expr))() } 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 b9cc25817d2f3..8570e4640feea 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 @@ -566,8 +566,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * You can set the following CSV-specific options to deal with CSV files: *
    - *
  • `sep` (default `,`): sets a single character as a separator for each - * field and value.
  • + *
  • `sep` (default `,`): sets a separator for each field and value. This separator can be one + * or more characters.
  • *
  • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding * type.
  • *
  • `quote` (default `"`): sets a single character used for escaping quoted values where 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 edf0963e71e81..68127c27a8cc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -26,8 +26,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsWrite, TableCatalog, TableProvider, V1Table} import org.apache.spark.sql.connector.catalog.TableCapability._ @@ -67,7 +66,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def mode(saveMode: SaveMode): DataFrameWriter[T] = { - this.mode = Some(saveMode) + this.mode = saveMode this } @@ -87,10 +86,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { 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'.") + case "error" | "errorifexists" | "default" => mode(SaveMode.ErrorIfExists) + case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. Accepted " + + "save modes are 'overwrite', 'append', 'ignore', 'error', 'errorifexists', 'default'.") } } @@ -267,7 +265,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { "if partition columns are specified.") } lazy val relation = DataSourceV2Relation.create(table, dsOptions) - modeForDSV2 match { + mode match { case SaveMode.Append => runCommand(df.sparkSession, "save") { AppendData.byName(relation, df.logicalPlan, extraOptions.toMap) @@ -308,7 +306,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { sparkSession = df.sparkSession, className = source, partitionColumns = partitioningColumns.getOrElse(Nil), - options = extraOptions.toMap).planForWriting(modeForDSV1, df.logicalPlan) + options = extraOptions.toMap).planForWriting(mode, df.logicalPlan) } } @@ -319,6 +317,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @note Unlike `saveAsTable`, `insertInto` ignores the column names and just uses position-based * resolution. For example: * + * @note SaveMode.ErrorIfExists and SaveMode.Ignore behave as SaveMode.Append in `insertInto` as + * `insertInto` is not a table creating operation. + * * {{{ * scala> Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("t1") * scala> Seq((3, 4)).toDF("j", "i").write.insertInto("t1") @@ -340,6 +341,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def insertInto(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ assertNotBucketed("insertInto") @@ -353,14 +355,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val sessionCatalog = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case CatalogObjectIdentifier(Some(catalog), ident) => + case CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog) => insertInto(catalog, ident) - case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 => - insertInto(sessionCatalog, ident) + case CatalogObjectIdentifier(catalog, ident) + if isSessionCatalog(catalog) && canUseV2 && ident.namespace().length <= 1 => + insertInto(catalog, ident) case AsTableIdentifier(tableIdentifier) => insertInto(tableIdentifier) @@ -380,8 +382,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { DataSourceV2Relation.create(t) } - val command = modeForDSV2 match { - case SaveMode.Append => + val command = mode match { + case SaveMode.Append | SaveMode.ErrorIfExists | SaveMode.Ignore => AppendData.byPosition(table, df.logicalPlan, extraOptions.toMap) case SaveMode.Overwrite => @@ -394,10 +396,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } else { OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true), extraOptions.toMap) } - - case other => - throw new AnalysisException(s"insertInto does not support $other mode, " + - s"please use Append or Overwrite mode instead.") } runCommand(df.sparkSession, "insertInto") { @@ -411,7 +409,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { table = UnresolvedRelation(tableIdent), partitionSpec = Map.empty[String, Option[String]], query = df.logicalPlan, - overwrite = modeForDSV1 == SaveMode.Overwrite, + overwrite = mode == SaveMode.Overwrite, ifPartitionNotExists = false) } } @@ -483,19 +481,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def saveAsTable(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val sessionCatalog = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case CatalogObjectIdentifier(Some(catalog), ident) => - saveAsTable(catalog.asTableCatalog, ident, modeForDSV2) + case CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog) => + saveAsTable(catalog.asTableCatalog, ident) - case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 => - // We pass in the modeForDSV1, as using the V2 session catalog should maintain compatibility - // for now. - saveAsTable(sessionCatalog.asTableCatalog, ident, modeForDSV1) + case CatalogObjectIdentifier(catalog, ident) + if isSessionCatalog(catalog) && canUseV2 && ident.namespace().length <= 1 => + saveAsTable(catalog.asTableCatalog, ident) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) @@ -507,7 +504,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } - private def saveAsTable(catalog: TableCatalog, ident: Identifier, mode: SaveMode): Unit = { + private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = { val partitioning = partitioningColumns.map { colNames => colNames.map(name => IdentityTransform(FieldReference(name))) }.getOrElse(Seq.empty[Transform]) @@ -568,7 +565,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val tableIdentWithDB = tableIdent.copy(database = Some(db)) val tableName = tableIdentWithDB.unquotedString - (tableExists, modeForDSV1) match { + (tableExists, mode) match { case (true, SaveMode.Ignore) => // Do nothing @@ -624,7 +621,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { bucketSpec = getBucketSpec) runCommand(df.sparkSession, "saveAsTable")( - CreateTable(tableDesc, modeForDSV1, Some(df.logicalPlan))) + CreateTable(tableDesc, mode, Some(df.logicalPlan))) } /** @@ -690,6 +687,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `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.
  • + *
  • `ignoreNullFields` (default `true`): Whether to ignore null fields + * when generating JSON objects.
  • *
* * @since 1.4.0 @@ -830,10 +829,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { SQLExecution.withNewExecutionId(session, qe, Some(name))(qe.toRdd) } - private def modeForDSV1 = mode.getOrElse(SaveMode.ErrorIfExists) - - private def modeForDSV2 = mode.getOrElse(SaveMode.Append) - private def lookupV2Provider(): Option[TableProvider] = { DataSource.lookupDataSourceV2(source, df.sparkSession.sessionState.conf) match { // TODO(SPARK-28396): File source v2 write path is currently broken. @@ -848,7 +843,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName - private var mode: Option[SaveMode] = None + private var mode: SaveMode = SaveMode.ErrorIfExists private val extraOptions = new scala.collection.mutable.HashMap[String, String] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index d3ab33919a7bb..9d3ce6fde20a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -51,13 +51,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) private val (catalog, identifier) = { - val CatalogObjectIdentifier(maybeCatalog, identifier) = tableName - val catalog = maybeCatalog.orElse(catalogManager.currentCatalog.map(catalogManager.catalog)) - .getOrElse(throw new AnalysisException( - s"No catalog specified for table ${identifier.quoted} and no default v2 catalog is set")) - .asTableCatalog - - (catalog, identifier) + val CatalogObjectIdentifier(catalog, identifier) = tableName + (catalog.asTableCatalog, identifier) } private val logicalPlan = df.queryExecution.logical @@ -93,7 +88,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) this } - override def tableProperty(property: String, value: String): DataFrameWriterV2[T] = { + override def tableProperty(property: String, value: String): CreateTableWriter[T] = { this.properties.put(property, value) this } 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 9a2d80030ee66..5f6e0a82be4ce 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 @@ -46,6 +46,7 @@ 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.trees.TreeNodeTag +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ @@ -254,10 +255,16 @@ class Dataset[T] private[sql]( @transient lazy val sqlContext: SQLContext = sparkSession.sqlContext private[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolveQuoted(colName, sparkSession.sessionState.analyzer.resolver) + val resolver = sparkSession.sessionState.analyzer.resolver + queryExecution.analyzed.resolveQuoted(colName, resolver) .getOrElse { - throw new AnalysisException( - s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") + val fields = schema.fieldNames + val extraMsg = if (fields.exists(resolver(_, colName))) { + s"; did you mean to quote the `$colName` column?" + } else "" + val fieldsStr = fields.mkString(", ") + val errorMsg = s"""Cannot resolve column name "$colName" among (${fieldsStr})${extraMsg}""" + throw new AnalysisException(errorMsg) } } @@ -718,7 +725,7 @@ class Dataset[T] private[sql]( def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { val parsedDelay = try { - CalendarInterval.fromCaseInsensitiveString(delayThreshold) + IntervalUtils.fromString(delayThreshold) } catch { case e: IllegalArgumentException => throw new AnalysisException( 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 f6d13be0e89be..4d4731870700c 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 @@ -47,8 +47,8 @@ import org.apache.spark.sql.types.{NumericType, StructType} */ @Stable class RelationalGroupedDataset protected[sql]( - val df: DataFrame, - val groupingExprs: Seq[Expression], + private[sql] val df: DataFrame, + private[sql] val groupingExprs: Seq[Expression], groupType: RelationalGroupedDataset.GroupType) { private[this] def toDF(aggExprs: Seq[Expression]): DataFrame = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala new file mode 100644 index 0000000000000..f91686cb544c9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{AnalysisException, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} + +/** + * Resolves catalogs from the multi-part identifiers in SQL statements, and convert the statements + * to the corresponding v1 or v2 commands if the resolved catalog is the session catalog. + * + * We can remove this rule once we implement all the catalog functionality in `V2SessionCatalog`. + */ +class ResolveSessionCatalog( + val catalogManager: CatalogManager, + conf: SQLConf, + isView: Seq[String] => Boolean) + extends Rule[LogicalPlan] with LookupCatalog { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case AlterTableAddColumnsStatement( + nameParts @ SessionCatalog(catalog, tableName), cols) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + cols.foreach(c => assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand")) + AlterTableAddColumnsCommand(tableName.asTableIdentifier, cols.map(convertToStructField)) + }.getOrElse { + val changes = cols.map { col => + TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + } + createAlterTable(nameParts, catalog, tableName, changes) + } + + case AlterTableAlterColumnStatement( + nameParts @ SessionCatalog(catalog, tableName), colName, dataType, comment) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + // TODO(SPARK-29353): we should fallback to the v1 `AlterTableChangeColumnCommand`. + throw new AnalysisException("ALTER COLUMN is only supported with v2 tables.") + }.getOrElse { + val typeChange = dataType.map { newDataType => + TableChange.updateColumnType(colName.toArray, newDataType, true) + } + val commentChange = comment.map { newComment => + TableChange.updateColumnComment(colName.toArray, newComment) + } + createAlterTable(nameParts, catalog, tableName, typeChange.toSeq ++ commentChange) + } + + case AlterTableRenameColumnStatement( + nameParts @ SessionCatalog(catalog, tableName), col, newName) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + throw new AnalysisException("RENAME COLUMN is only supported with v2 tables.") + }.getOrElse { + val changes = Seq(TableChange.renameColumn(col.toArray, newName)) + createAlterTable(nameParts, catalog, tableName, changes) + } + + case AlterTableDropColumnsStatement( + nameParts @ SessionCatalog(catalog, tableName), cols) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + throw new AnalysisException("DROP COLUMN is only supported with v2 tables.") + }.getOrElse { + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + createAlterTable(nameParts, catalog, tableName, changes) + } + + case AlterTableSetPropertiesStatement( + nameParts @ SessionCatalog(catalog, tableName), props) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + AlterTableSetPropertiesCommand(tableName.asTableIdentifier, props, isView = false) + }.getOrElse { + val changes = props.map { case (key, value) => + TableChange.setProperty(key, value) + }.toSeq + createAlterTable(nameParts, catalog, tableName, changes) + } + + case AlterTableUnsetPropertiesStatement( + nameParts @ SessionCatalog(catalog, tableName), keys, ifExists) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + AlterTableUnsetPropertiesCommand( + tableName.asTableIdentifier, keys, ifExists, isView = false) + }.getOrElse { + val changes = keys.map(key => TableChange.removeProperty(key)) + createAlterTable(nameParts, catalog, tableName, changes) + } + + case AlterTableSetLocationStatement( + nameParts @ SessionCatalog(catalog, tableName), newLoc) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + AlterTableSetLocationCommand(tableName.asTableIdentifier, None, newLoc) + }.getOrElse { + val changes = Seq(TableChange.setProperty("location", newLoc)) + createAlterTable(nameParts, catalog, tableName, changes) + } + + // ALTER VIEW should always use v1 command if the resolved catalog is session catalog. + case AlterViewSetPropertiesStatement(SessionCatalog(catalog, tableName), props) => + AlterTableSetPropertiesCommand(tableName.asTableIdentifier, props, isView = true) + + case AlterViewUnsetPropertiesStatement(SessionCatalog(catalog, tableName), keys, ifExists) => + AlterTableUnsetPropertiesCommand(tableName.asTableIdentifier, keys, ifExists, isView = true) + + case DeleteFromStatement( + nameParts @ SessionCatalog(catalog, tableName), tableAlias, condition) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + throw new AnalysisException("DELETE FROM is only supported with v2 tables.") + }.getOrElse { + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + val aliased = tableAlias.map(SubqueryAlias(_, r)).getOrElse(r) + DeleteFromTable(aliased, condition) + } + + case DescribeTableStatement( + nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + DescribeTableCommand(tableName.asTableIdentifier, partitionSpec, isExtended) + }.getOrElse { + // The v1 `DescribeTableCommand` can describe view as well. + if (isView(tableName)) { + DescribeTableCommand(tableName.asTableIdentifier, partitionSpec, isExtended) + } else { + if (partitionSpec.nonEmpty) { + throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.") + } + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + DescribeTable(r, isExtended) + } + } + + case DescribeColumnStatement(SessionCatalog(catalog, tableName), colNameParts, isExtended) => + loadTable(catalog, tableName.asIdentifier).collect { + case v1Table: V1Table => + DescribeColumnCommand(tableName.asTableIdentifier, colNameParts, isExtended) + }.getOrElse { + if (isView(tableName)) { + DescribeColumnCommand(tableName.asTableIdentifier, colNameParts, isExtended) + } else { + throw new AnalysisException("Describing columns is not supported for v2 tables.") + } + } + + // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the + // session catalog and the table provider is not v2. + case c @ CreateTableStatement( + SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + if (!isV2Provider(c.provider)) { + val tableDesc = buildCatalogTable(c.tableName.asTableIdentifier, c.tableSchema, + c.partitioning, c.bucketSpec, c.properties, c.provider, c.options, c.location, + c.comment, c.ifNotExists) + val mode = if (c.ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists + CreateTable(tableDesc, mode, None) + } else { + CreateV2Table( + catalog.asTableCatalog, + tableName.asIdentifier, + c.tableSchema, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + ignoreIfExists = c.ifNotExists) + } + + case c @ CreateTableAsSelectStatement( + SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + if (!isV2Provider(c.provider)) { + val tableDesc = buildCatalogTable(c.tableName.asTableIdentifier, new StructType, + c.partitioning, c.bucketSpec, c.properties, c.provider, c.options, c.location, + c.comment, c.ifNotExists) + val mode = if (c.ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists + CreateTable(tableDesc, mode, Some(c.asSelect)) + } else { + CreateTableAsSelect( + catalog.asTableCatalog, + tableName.asIdentifier, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + c.asSelect, + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + writeOptions = c.options.filterKeys(_ != "path"), + ignoreIfExists = c.ifNotExists) + } + + case RefreshTableStatement(SessionCatalog(_, tableName)) => + RefreshTable(tableName.asTableIdentifier) + + // For REPLACE TABLE [AS SELECT], we should fail if the catalog is resolved to the + // session catalog and the table provider is not v2. + case c @ ReplaceTableStatement( + SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + if (!isV2Provider(c.provider)) { + throw new AnalysisException("REPLACE TABLE is only supported with v2 tables.") + } else { + ReplaceTable( + catalog.asTableCatalog, + tableName.asIdentifier, + c.tableSchema, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + orCreate = c.orCreate) + } + + case c @ ReplaceTableAsSelectStatement( + SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + if (!isV2Provider(c.provider)) { + throw new AnalysisException("REPLACE TABLE AS SELECT is only supported with v2 tables.") + } else { + ReplaceTableAsSelect( + catalog.asTableCatalog, + tableName.asIdentifier, + // convert the bucket spec and add it as a transform + c.partitioning ++ c.bucketSpec.map(_.asTransform), + c.asSelect, + convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), + writeOptions = c.options.filterKeys(_ != "path"), + orCreate = c.orCreate) + } + + case d @ DropTableStatement(SessionCatalog(catalog, tableName), ifExists, purge) => + DropTableCommand(d.tableName.asTableIdentifier, ifExists, isView = false, purge = purge) + + case DropViewStatement(SessionCatalog(catalog, viewName), ifExists) => + DropTableCommand(viewName.asTableIdentifier, ifExists, isView = true, purge = false) + + case c @ CreateNamespaceStatement(SessionCatalog(catalog, nameParts), _, _) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + + val comment = c.properties.get(CreateNamespaceStatement.COMMENT_PROPERTY_KEY) + val location = c.properties.get(CreateNamespaceStatement.LOCATION_PROPERTY_KEY) + val newProperties = c.properties - + CreateNamespaceStatement.COMMENT_PROPERTY_KEY - + CreateNamespaceStatement.LOCATION_PROPERTY_KEY + CreateDatabaseCommand(nameParts.head, c.ifNotExists, location, comment, newProperties) + + case ShowTablesStatement(Some(SessionCatalog(catalog, nameParts)), pattern) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + ShowTablesCommand(Some(nameParts.head), pattern) + + case ShowTablesStatement(None, pattern) if isSessionCatalog(currentCatalog) => + ShowTablesCommand(None, pattern) + + case AnalyzeTableStatement(tableName, partitionSpec, noScan) => + val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") + if (partitionSpec.isEmpty) { + AnalyzeTableCommand(v1TableName.asTableIdentifier, noScan) + } else { + AnalyzePartitionCommand(v1TableName.asTableIdentifier, partitionSpec, noScan) + } + + case AnalyzeColumnStatement(tableName, columnNames, allColumns) => + val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") + AnalyzeColumnCommand(v1TableName.asTableIdentifier, columnNames, allColumns) + + case RepairTableStatement(tableName) => + val v1TableName = parseV1Table(tableName, "MSCK REPAIR TABLE") + AlterTableRecoverPartitionsCommand( + v1TableName.asTableIdentifier, + "MSCK REPAIR TABLE") + + case CacheTableStatement(tableName, plan, isLazy, options) => + val v1TableName = parseV1Table(tableName, "CACHE TABLE") + CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) + + case UncacheTableStatement(tableName, ifExists) => + val v1TableName = parseV1Table(tableName, "UNCACHE TABLE") + UncacheTableCommand(v1TableName.asTableIdentifier, ifExists) + + case TruncateTableStatement(tableName, partitionSpec) => + val v1TableName = parseV1Table(tableName, "TRUNCATE TABLE") + TruncateTableCommand( + v1TableName.asTableIdentifier, + partitionSpec) + + case ShowPartitionsStatement(tableName, partitionSpec) => + val v1TableName = parseV1Table(tableName, "SHOW PARTITIONS") + ShowPartitionsCommand( + v1TableName.asTableIdentifier, + partitionSpec) + } + + private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { + val CatalogAndIdentifierParts(catalog, parts) = tableName + if (!isSessionCatalog(catalog)) { + throw new AnalysisException(s"$sql is only supported with v1 tables.") + } + parts + } + + 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) + } + + object SessionCatalog { + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { + case CatalogAndIdentifierParts(catalog, parts) if isSessionCatalog(catalog) => + Some(catalog -> parts) + case _ => None + } + } + + private def assertTopLevelColumn(colName: Seq[String], command: String): Unit = { + if (colName.length > 1) { + throw new AnalysisException(s"$command does not support nested column: ${colName.quoted}") + } + } + + 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()) + } + + private def isV2Provider(provider: String): Boolean = { + DataSource.lookupDataSourceV2(provider, conf) match { + // TODO(SPARK-28396): Currently file source v2 can't work with tables. + case Some(_: FileDataSourceV2) => false + case Some(_) => true + case _ => false + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala new file mode 100644 index 0000000000000..fa5ba1a691cd6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.mutable + +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSeq, BindReferences, Expression, InterpretedMutableProjection, InterpretedUnsafeProjection, JoinedRow, MutableProjection, NamedExpression, Projection, SpecificInternalRow} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, DeclarativeAggregate, ImperativeAggregate, NoOp, TypedImperativeAggregate} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.util.AccumulatorV2 + +/** + * Accumulator that computes a global aggregate. + */ +class AggregatingAccumulator private( + bufferSchema: Seq[DataType], + initialValues: Seq[Expression], + updateExpressions: Seq[Expression], + @transient private val mergeExpressions: Seq[Expression], + @transient private val resultExpressions: Seq[Expression], + imperatives: Array[ImperativeAggregate], + typedImperatives: Array[TypedImperativeAggregate[_]], + @transient private val conf: SQLConf) + extends AccumulatorV2[InternalRow, InternalRow] { + assert(bufferSchema.size == initialValues.size) + assert(bufferSchema.size == updateExpressions.size) + assert(mergeExpressions == null || bufferSchema.size == mergeExpressions.size) + + private[this] var joinedRow: JoinedRow = _ + + private var buffer: SpecificInternalRow = _ + + private def createBuffer(): SpecificInternalRow = { + val buffer = new SpecificInternalRow(bufferSchema) + + // Initialize the buffer. Note that we do not use a code generated projection here because + // generating and compiling a projection is probably more expensive than using an interpreted + // projection. + InterpretedMutableProjection.createProjection(initialValues) + .target(buffer) + .apply(InternalRow.empty) + imperatives.foreach(_.initialize(buffer)) + typedImperatives.foreach(_.initialize(buffer)) + buffer + } + + private def getOrCreateBuffer(): SpecificInternalRow = { + if (buffer == null) { + buffer = createBuffer() + + // Create the joined row and set the buffer as its 'left' row. + joinedRow = new JoinedRow() + joinedRow.withLeft(buffer) + } + buffer + } + + private def initializeProjection[T <: Projection](projection: T): T = { + projection.initialize(TaskContext.getPartitionId()) + projection + } + + @transient + private[this] lazy val updateProjection = initializeProjection { + MutableProjection.create(updateExpressions) + } + + @transient + private[this] lazy val mergeProjection = initializeProjection { + InterpretedMutableProjection.createProjection(mergeExpressions) + } + + @transient + private[this] lazy val resultProjection = initializeProjection { + InterpretedUnsafeProjection.createProjection(resultExpressions) + } + + /** + * Driver side operations like `merge` and `value` are executed in the DAGScheduler thread. This + * thread does not have a SQL configuration so we attach our own here. Note that we can't (and + * shouldn't) call `merge` or `value` on an accumulator originating from an executor so we just + * return a default value here. + */ + private[this] def withSQLConf[T](default: => T)(body: => T): T = { + if (conf != null) { + SQLConf.withExistingConf(conf)(body) + } else { + default + } + } + + override def reset(): Unit = { + buffer = null + joinedRow = null + } + + override def isZero: Boolean = buffer == null + + override def copyAndReset(): AggregatingAccumulator = { + new AggregatingAccumulator( + bufferSchema, + initialValues, + updateExpressions, + mergeExpressions, + resultExpressions, + imperatives, + typedImperatives, + conf) + } + + override def copy(): AggregatingAccumulator = { + val copy = copyAndReset() + copy.merge(this) + copy + } + + override def add(v: InternalRow): Unit = { + val buffer = getOrCreateBuffer() + updateProjection.target(buffer)(joinedRow.withRight(v)) + var i = 0 + while (i < imperatives.length) { + imperatives(i).update(buffer, v) + i += 1 + } + i = 0 + while (i < typedImperatives.length) { + typedImperatives(i).update(buffer, v) + i += 1 + } + } + + override def merge(other: AccumulatorV2[InternalRow, InternalRow]): Unit = withSQLConf(()) { + if (!other.isZero) { + other match { + case agg: AggregatingAccumulator => + val buffer = getOrCreateBuffer() + val otherBuffer = agg.buffer + mergeProjection.target(buffer)(joinedRow.withRight(otherBuffer)) + var i = 0 + while (i < imperatives.length) { + imperatives(i).merge(buffer, otherBuffer) + i += 1 + } + i = 0 + while (i < typedImperatives.length) { + typedImperatives(i).mergeBuffersObjects(buffer, otherBuffer) + i += 1 + } + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + } + } + + override def value: InternalRow = withSQLConf(InternalRow.empty) { + // Either use the existing buffer or create a temporary one. + val input = if (!isZero) { + buffer + } else { + // Create a temporary buffer because we want to avoid changing the state of the accumulator + // here, which would happen if we called getOrCreateBuffer(). This is relatively expensive to + // do but it should be no problem since this method is supposed to be called rarely (once per + // query execution). + createBuffer() + } + resultProjection(input) + } + + + /** + * Get the output schema of the aggregating accumulator. + */ + lazy val schema: StructType = { + StructType(resultExpressions.zipWithIndex.map { + case (e: NamedExpression, _) => StructField(e.name, e.dataType, e.nullable, e.metadata) + case (e, i) => StructField(s"c_$i", e.dataType, e.nullable) + }) + } +} + +object AggregatingAccumulator { + /** + * Create an aggregating accumulator for the given functions and input schema. + */ + def apply(functions: Seq[Expression], inputAttributes: Seq[Attribute]): AggregatingAccumulator = { + // There are a couple of things happening here: + // - Collect the schema's of the aggregate and input aggregate buffers. These are needed to bind + // the expressions which will be done when we create the accumulator. + // - Collect the initialValues, update and merge expressions for declarative aggregate + // functions. + // - Bind and Collect the imperative aggregate functions. Note that we insert NoOps into the + // (declarative) initialValues, update and merge expression buffers to keep these aligned with + // the aggregate buffer. + // - Build the result expressions. + val aggBufferAttributes = mutable.Buffer.empty[AttributeReference] + val inputAggBufferAttributes = mutable.Buffer.empty[AttributeReference] + val initialValues = mutable.Buffer.empty[Expression] + val updateExpressions = mutable.Buffer.empty[Expression] + val mergeExpressions = mutable.Buffer.empty[Expression] + val imperatives = mutable.Buffer.empty[ImperativeAggregate] + val typedImperatives = mutable.Buffer.empty[TypedImperativeAggregate[_]] + val inputAttributeSeq: AttributeSeq = inputAttributes + val resultExpressions = functions.map(_.transform { + case AggregateExpression(agg: DeclarativeAggregate, _, _, _) => + aggBufferAttributes ++= agg.aggBufferAttributes + inputAggBufferAttributes ++= agg.inputAggBufferAttributes + initialValues ++= agg.initialValues + updateExpressions ++= agg.updateExpressions + mergeExpressions ++= agg.mergeExpressions + agg.evaluateExpression + case AggregateExpression(agg: ImperativeAggregate, _, _, _) => + val imperative = BindReferences.bindReference(agg + .withNewMutableAggBufferOffset(aggBufferAttributes.size) + .withNewInputAggBufferOffset(inputAggBufferAttributes.size), + inputAttributeSeq) + imperative match { + case typedImperative: TypedImperativeAggregate[_] => + typedImperatives += typedImperative + case _ => + imperatives += imperative + } + aggBufferAttributes ++= imperative.aggBufferAttributes + inputAggBufferAttributes ++= agg.inputAggBufferAttributes + val noOps = Seq.fill(imperative.aggBufferAttributes.size)(NoOp) + initialValues ++= noOps + updateExpressions ++= noOps + mergeExpressions ++= noOps + imperative + }) + + val updateAttrSeq: AttributeSeq = aggBufferAttributes ++ inputAttributes + val mergeAttrSeq: AttributeSeq = aggBufferAttributes ++ inputAggBufferAttributes + val aggBufferAttributesSeq: AttributeSeq = aggBufferAttributes + + // Create the accumulator. + new AggregatingAccumulator( + aggBufferAttributes.map(_.dataType), + initialValues, + updateExpressions.map(BindReferences.bindReference(_, updateAttrSeq)), + mergeExpressions.map(BindReferences.bindReference(_, mergeAttrSeq)), + resultExpressions.map(BindReferences.bindReference(_, aggBufferAttributesSeq)), + imperatives.toArray, + typedImperatives.toArray, + SQLConf.get) + } +} 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 4a87049ac292b..b0fe4b741479f 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 @@ -65,10 +65,26 @@ trait DataSourceScanExec extends LeafExecNode { s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr") } + override def verboseStringWithOperatorId(): String = { + val metadataStr = metadata.toSeq.sorted.filterNot { + case (_, value) if (value.isEmpty || value.equals("[]")) => true + case (key, _) if (key.equals("DataFilters") || key.equals("Format")) => true + case (_, _) => false + }.map { + case (key, value) => s"$key: ${redact(value)}" + } + + s""" + |(${ExplainUtils.getOpId(this)}) $nodeName ${ExplainUtils.getCodegenId(this)} + |Output: ${producedAttributes.mkString("[", ", ", "]")} + |${metadataStr.mkString("\n")} + """.stripMargin + } + /** * Shorthand for calling redactString() without specifying redacting rules */ - private def redact(text: String): String = { + protected def redact(text: String): String = { Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, text) } @@ -342,6 +358,31 @@ case class FileSourceScanExec( withSelectedBucketsCount } + override def verboseStringWithOperatorId(): String = { + val metadataStr = metadata.toSeq.sorted.filterNot { + case (_, value) if (value.isEmpty || value.equals("[]")) => true + case (key, _) if (key.equals("DataFilters") || key.equals("Format")) => true + case (_, _) => false + }.map { + case (key, _) if (key.equals("Location")) => + val location = relation.location + val numPaths = location.rootPaths.length + val abbreviatedLoaction = if (numPaths <= 1) { + location.rootPaths.mkString("[", ", ", "]") + } else { + "[" + location.rootPaths.head + s", ... ${numPaths - 1} entries]" + } + s"$key: ${location.getClass.getSimpleName} ${redact(abbreviatedLoaction)}" + case (key, value) => s"$key: ${redact(value)}" + } + + s""" + |(${ExplainUtils.getOpId(this)}) $nodeName ${ExplainUtils.getCodegenId(this)} + |Output: ${producedAttributes.mkString("[", ", ", "]")} + |${metadataStr.mkString("\n")} + """.stripMargin + } + lazy val inputRDD: RDD[InternalRow] = { val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala index fc384fe117caf..d4fe272f8c95f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala @@ -193,14 +193,14 @@ object ExplainUtils { subqueries: ArrayBuffer[(SparkPlan, Expression, BaseSubqueryExec)]): Unit = { plan.foreach { case p: SparkPlan => - p.expressions.flatMap(_.collect { + p.expressions.foreach (_.collect { case e: PlanExpression[_] => e.plan match { case s: BaseSubqueryExec => subqueries += ((p, e, s)) getSubqueries(s, subqueries) + case _ => } - case other => }) } } 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 f294a56c60e26..f9394473d06e0 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 @@ -137,7 +137,11 @@ class QueryExecution( val concat = new PlanStringConcat() concat.append("== Physical Plan ==\n") if (formatted) { - ExplainUtils.processPlan(executedPlan, concat.append) + try { + ExplainUtils.processPlan(executedPlan, concat.append) + } catch { + case e: AnalysisException => concat.append(e.toString) + } } else { QueryPlan.append(executedPlan, concat.append, verbose = false, addSuffix = false) } @@ -176,8 +180,11 @@ class QueryExecution( val maxFields = SQLConf.get.maxToStringFields // trigger to compute stats for logical plans - optimizedPlan.stats - + try { + optimizedPlan.stats + } catch { + case e: AnalysisException => concat.append(e.toString + "\n") + } // only show optimized logical plan and physical plan concat.append("== Optimized Logical Plan ==\n") QueryPlan.append(optimizedPlan, concat.append, verbose = true, addSuffix = true, maxFields) 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 079ff25fcb67e..4c19f95796d04 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 @@ -21,8 +21,10 @@ import java.util.Arrays import org.apache.spark._ import org.apache.spark.rdd.RDD +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} +import org.apache.spark.sql.internal.SQLConf /** * The [[Partition]] used by [[ShuffledRowRDD]]. A post-shuffle partition @@ -117,6 +119,11 @@ class ShuffledRowRDD( specifiedPartitionStartIndices: Option[Array[Int]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { + if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) { + dependency.rdd.context.setLocalProperty( + SortShuffleManager.FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED_KEY, "true") + } + private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions private[this] val partitionStartIndices: Array[Int] = specifiedPartitionStartIndices match { @@ -172,7 +179,7 @@ class ShuffledRowRDD( reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) } - override def clearDependencies() { + override def clearDependencies(): Unit = { super.clearDependencies() dependency = null } 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 0a955d6a75235..32d21d05e5f73 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 @@ -62,6 +62,14 @@ case class SortExec( "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")) + private[sql] var rowSorter: UnsafeExternalRowSorter = _ + + /** + * This method gets invoked only once for each SortExec instance to initialize an + * UnsafeExternalRowSorter, both `plan.execute` and code generation are using it. + * In the code generation code path, we need to call this function outside the class so we + * should make it public. + */ def createSorter(): UnsafeExternalRowSorter = { val ordering = newOrdering(sortOrder, output) @@ -87,13 +95,13 @@ case class SortExec( } val pageSize = SparkEnv.get.memoryManager.pageSizeBytes - val sorter = UnsafeExternalRowSorter.create( + rowSorter = UnsafeExternalRowSorter.create( schema, ordering, prefixComparator, prefixComputer, pageSize, canUseRadixSort) if (testSpillFrequency > 0) { - sorter.setTestSpillFrequency(testSpillFrequency) + rowSorter.setTestSpillFrequency(testSpillFrequency) } - sorter + rowSorter } protected override def doExecute(): RDD[InternalRow] = { @@ -181,4 +189,17 @@ case class SortExec( |$sorterVariable.insertRow((UnsafeRow)${row.value}); """.stripMargin } + + /** + * In SortExec, we overwrites cleanupResources to close UnsafeExternalRowSorter. + */ + override protected[sql] def cleanupResources(): Unit = { + if (rowSorter != null) { + // There's possible for rowSorter is null here, for example, in the scenario of empty + // iterator in the current task, the downstream physical node(like SortMergeJoinExec) will + // trigger cleanupResources before rowSorter initialized in createSorter. + rowSorter.cleanupResources() + } + super.cleanupResources() + } } 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 98d6be0374da7..4a8b56faddf7e 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 @@ -20,15 +20,17 @@ 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._ +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning} import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs} class SparkOptimizer( + catalogManager: CatalogManager, catalog: SessionCatalog, experimentalMethods: ExperimentalMethods) - extends Optimizer(catalog) { + extends Optimizer(catalogManager) { override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ 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 b4cdf9e16b7e5..125f76282e3df 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 @@ -507,6 +507,15 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } newOrdering(order, Seq.empty) } + + /** + * Cleans up the resources used by the physical operator (if any). In general, all the resources + * should be cleaned up when the task finishes but operators like SortMergeJoinExec and LimitExec + * may want eager cleanup to free up tight resources (e.g., memory). + */ + protected[sql] def cleanupResources(): Unit = { + children.foreach(_.cleanupResources()) + } } trait LeafExecNode extends SparkPlan { 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 8c7752c4bb742..459311df22d23 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,7 +18,7 @@ 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.adaptive.{AdaptiveSparkPlanExec, LocalShuffleReaderExec, QueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.sql.internal.SQLConf @@ -56,6 +56,7 @@ private[execution] object SparkPlanInfo { case ReusedSubqueryExec(child) => child :: Nil case a: AdaptiveSparkPlanExec => a.executedPlan :: Nil case stage: QueryStageExec => stage.plan :: Nil + case localReader: LocalShuffleReaderExec => localReader.child :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => 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 83cdc7a978a9a..aef0a2d2e5951 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 @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.StructType /** * Concrete parser for Spark SQL statements. */ -class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser { +class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { val astBuilder = new SparkSqlAstBuilder(conf) private val substitutor = new VariableSubstitution(conf) @@ -89,62 +89,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ResetCommand } - /** - * Create an [[AnalyzeTableCommand]] command, or an [[AnalyzePartitionCommand]] - * or an [[AnalyzeColumnCommand]] command. - * Example SQL for analyzing a table or a set of partitions : - * {{{ - * ANALYZE TABLE [db_name.]tablename [PARTITION (partcol1[=val1], partcol2[=val2], ...)] - * COMPUTE STATISTICS [NOSCAN]; - * }}} - * - * Example SQL for analyzing columns : - * {{{ - * ANALYZE TABLE [db_name.]tablename COMPUTE STATISTICS FOR COLUMNS column1, column2; - * }}} - * - * Example SQL for analyzing all columns of a table: - * {{{ - * ANALYZE TABLE [db_name.]tablename COMPUTE STATISTICS FOR ALL COLUMNS; - * }}} - */ - override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) { - def checkPartitionSpec(): Unit = { - if (ctx.partitionSpec != null) { - logWarning("Partition specification is ignored when collecting column statistics: " + - ctx.partitionSpec.getText) - } - } - if (ctx.identifier != null && - ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { - throw new ParseException(s"Expected `NOSCAN` instead of `${ctx.identifier.getText}`", ctx) - } - - val table = visitTableIdentifier(ctx.tableIdentifier) - if (ctx.ALL() != null) { - checkPartitionSpec() - AnalyzeColumnCommand(table, None, allColumns = true) - } else if (ctx.identifierSeq() == null) { - if (ctx.partitionSpec != null) { - AnalyzePartitionCommand(table, visitPartitionSpec(ctx.partitionSpec), - noscan = ctx.identifier != null) - } else { - AnalyzeTableCommand(table, noscan = ctx.identifier != null) - } - } else { - checkPartitionSpec() - AnalyzeColumnCommand(table, - Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) - } - } - - /** - * Create a [[SetDatabaseCommand]] logical plan. - */ - override def visitUse(ctx: UseContext): LogicalPlan = withOrigin(ctx) { - SetDatabaseCommand(ctx.db.getText) - } - /** * Create a [[ShowTablesCommand]] logical plan. * Example SQL : @@ -162,17 +106,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { partitionSpec = partitionSpec) } - /** - * Create a [[ShowDatabasesCommand]] logical plan. - * Example SQL: - * {{{ - * SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards']; - * }}} - */ - override def visitShowDatabases(ctx: ShowDatabasesContext): LogicalPlan = withOrigin(ctx) { - ShowDatabasesCommand(Option(ctx.pattern).map(string)) - } - /** * A command for users to list the properties for a table. If propertyKey is specified, the value * for the propertyKey is returned. If propertyKey is not specified, all the keys and their @@ -202,23 +135,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier)) } - /** - * A command for users to list the partition names of a table. If partition spec is specified, - * partitions that match the spec are returned. Otherwise an empty result set is returned. - * - * This function creates a [[ShowPartitionsCommand]] logical plan - * - * The syntax of using this command in SQL is: - * {{{ - * SHOW PARTITIONS table_identifier [partition_spec]; - * }}} - */ - override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { - val table = visitTableIdentifier(ctx.tableIdentifier) - val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ShowPartitionsCommand(table, partitionKeys) - } - /** * Creates a [[ShowCreateTableCommand]] */ @@ -227,13 +143,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ShowCreateTableCommand(table) } - /** - * Create a [[RefreshTable]] logical plan. - */ - override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { - RefreshTable(visitTableIdentifier(ctx.tableIdentifier)) - } - /** * Create a [[RefreshResource]] logical plan. */ @@ -256,28 +165,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { unquotedPath } - /** - * Create a [[CacheTableCommand]] logical plan. - */ - override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { - val query = Option(ctx.query).map(plan) - val tableIdent = visitTableIdentifier(ctx.tableIdentifier) - if (query.isDefined && tableIdent.database.isDefined) { - val database = tableIdent.database.get - throw new ParseException(s"It is not allowed to add database prefix `$database` to " + - s"the table name in CACHE TABLE AS SELECT", ctx) - } - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - CacheTableCommand(tableIdent, query, ctx.LAZY != null, options) - } - - /** - * Create an [[UncacheTableCommand]] logical plan. - */ - override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { - UncacheTableCommand(visitTableIdentifier(ctx.tableIdentifier), ctx.EXISTS != null) - } - /** * Create a [[ClearCacheCommand]] logical plan. */ @@ -414,73 +301,34 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } /** - * Create a [[TruncateTableCommand]] command. - * - * For example: - * {{{ - * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] - * }}} - */ - override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { - TruncateTableCommand( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) - } - - /** - * Create a [[AlterTableRecoverPartitionsCommand]] command. - * - * For example: - * {{{ - * MSCK REPAIR TABLE tablename - * }}} - */ - override def visitRepairTable(ctx: RepairTableContext): LogicalPlan = withOrigin(ctx) { - AlterTableRecoverPartitionsCommand( - visitTableIdentifier(ctx.tableIdentifier), - "MSCK REPAIR TABLE") - } - - /** - * Create a [[CreateDatabaseCommand]] command. + * Create an [[AlterDatabasePropertiesCommand]] command. * * For example: * {{{ - * 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, ...)] + * ALTER (DATABASE|SCHEMA) database SET DBPROPERTIES (property_name=property_value, ...); * }}} */ - 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( + override def visitSetDatabaseProperties( + ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { + AlterDatabasePropertiesCommand( ctx.db.getText, - ctx.EXISTS != null, - ctx.locationSpec.asScala.headOption.map(visitLocationSpec), - Option(ctx.comment).map(string), - ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues).getOrElse(Map.empty)) + visitPropertyKeyValues(ctx.tablePropertyList)) } /** - * Create an [[AlterDatabasePropertiesCommand]] command. + * Create an [[AlterDatabaseSetLocationCommand]] command. * * For example: * {{{ - * ALTER (DATABASE|SCHEMA) database SET DBPROPERTIES (property_name=property_value, ...); + * ALTER (DATABASE|SCHEMA) database SET LOCATION path; * }}} */ - override def visitSetDatabaseProperties( - ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterDatabasePropertiesCommand( + override def visitSetDatabaseLocation( + ctx: SetDatabaseLocationContext): LogicalPlan = withOrigin(ctx) { + AlterDatabaseSetLocationCommand( ctx.db.getText, - visitPropertyKeyValues(ctx.tablePropertyList)) + visitLocationSpec(ctx.locationSpec) + ) } /** 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 a2f45898d273f..6e43c9b8bd80b 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 @@ -745,6 +745,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil + case _: UpdateTable => + throw new UnsupportedOperationException(s"UPDATE TABLE is not supported temporarily.") case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 524cacc11484c..f01947d8f5ed6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -84,6 +84,7 @@ case class AdaptiveSparkPlanExec( // plan should reach a final status of query stages (i.e., no more addition or removal of // Exchange nodes) after running these rules. private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( + OptimizeLocalShuffleReader(conf), ensureRequirements ) @@ -91,6 +92,15 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, subqueryCache), + + // When adding local shuffle readers in 'OptimizeLocalShuffleReader`, we revert all the local + // readers if additional shuffles are introduced. This may be too conservative: maybe there is + // only one local reader that introduces shuffle, and we can still keep other local readers. + // Here we re-execute this rule with the sub-plan-tree of a query stage, to make sure necessary + // local readers are added before executing the query stage. + // This rule must be executed before `ReduceNumShufflePartitions`, as local shuffle readers + // can't change number of partitions. + OptimizeLocalShuffleReader(conf), ReduceNumShufflePartitions(conf), ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, session.sessionState.columnarRules), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 0ec8710e4db43..94e66b0c3a430 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -125,6 +125,7 @@ trait AdaptiveSparkPlanHelper { private def allChildren(p: SparkPlan): Seq[SparkPlan] = p match { case a: AdaptiveSparkPlanExec => Seq(a.executedPlan) case s: QueryStageExec => Seq(s.plan) + case l: LocalShuffleReaderExec => Seq(l.child) case _ => p.children } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala new file mode 100644 index 0000000000000..5fccb5ce65783 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.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.execution.adaptive + +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 [[LocalShuffledRowRDD]]. A pre-shuffle partition + * (identified by `preShufflePartitionIndex`) contains a range of post-shuffle partitions + * (`startPostShufflePartitionIndex` to `endPostShufflePartitionIndex - 1`, inclusive). + */ +private final class LocalShuffledRowRDDPartition( + val preShufflePartitionIndex: Int) extends Partition { + override val index: Int = preShufflePartitionIndex +} + +/** + * This is a specialized version of [[org.apache.spark.sql.execution.ShuffledRowRDD]]. This is used + * in Spark SQL adaptive execution when a shuffle join is converted to broadcast join at runtime + * because the map output of one input table is small enough for broadcast. This RDD represents the + * data of another input table of the join that reads from shuffle. Each partition of the RDD reads + * the whole data from just one mapper output locally. So actually there is no data transferred + * from the network. + + * This RDD takes a [[ShuffleDependency]] (`dependency`). + * + * The `dependency` has the parent RDD of this RDD, which represents the dataset before shuffle + * (i.e. map output). Elements of this RDD are (partitionId, Row) pairs. + * Partition ids should be in the range [0, numPartitions - 1]. + * `dependency.partitioner.numPartitions` is the number of pre-shuffle partitions. (i.e. the number + * of partitions of the map output). The post-shuffle partition number is the same to the parent + * RDD's partition number. + */ +class LocalShuffledRowRDD( + var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + metrics: Map[String, SQLMetric]) + extends RDD[InternalRow](dependency.rdd.context, Nil) { + + private[this] val numReducers = dependency.partitioner.numPartitions + private[this] val numMappers = dependency.rdd.partitions.length + + override def getDependencies: Seq[Dependency[_]] = List(dependency) + + override def getPartitions: Array[Partition] = { + + Array.tabulate[Partition](numMappers) { i => + new LocalShuffledRowRDDPartition(i) + } + } + + override def getPreferredLocations(partition: Partition): Seq[String] = { + val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + tracker.getMapLocation(dependency, partition.index) + } + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val localRowPartition = split.asInstanceOf[LocalShuffledRowRDDPartition] + val mapIndex = localRowPartition.index + 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) + + val reader = SparkEnv.get.shuffleManager.getReaderForOneMapper( + dependency.shuffleHandle, + mapIndex, + 0, + numReducers, + context, + sqlMetricsReporter) + reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) + } + + override def clearDependencies() { + super.clearDependencies() + dependency = null + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala new file mode 100644 index 0000000000000..89e2813695a65 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.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.adaptive + +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.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight} +import org.apache.spark.sql.internal.SQLConf + +case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { + + def canUseLocalShuffleReaderLeft(join: BroadcastHashJoinExec): Boolean = { + join.buildSide == BuildRight && ShuffleQueryStageExec.isShuffleQueryStageExec(join.left) + } + + def canUseLocalShuffleReaderRight(join: BroadcastHashJoinExec): Boolean = { + join.buildSide == BuildLeft && ShuffleQueryStageExec.isShuffleQueryStageExec(join.right) + } + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { + return plan + } + + val optimizedPlan = plan.transformDown { + case join: BroadcastHashJoinExec if canUseLocalShuffleReaderRight(join) => + val localReader = LocalShuffleReaderExec(join.right.asInstanceOf[QueryStageExec]) + join.copy(right = localReader) + case join: BroadcastHashJoinExec if canUseLocalShuffleReaderLeft(join) => + val localReader = LocalShuffleReaderExec(join.left.asInstanceOf[QueryStageExec]) + join.copy(left = localReader) + } + + def numExchanges(plan: SparkPlan): Int = { + plan.collect { + case e: ShuffleExchangeExec => e + }.length + } + + val numExchangeBefore = numExchanges(EnsureRequirements(conf).apply(plan)) + val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) + + if (numExchangeAfter > numExchangeBefore) { + logDebug("OptimizeLocalShuffleReader rule is not applied due" + + " to additional shuffles will be introduced.") + plan + } else { + optimizedPlan + } + } +} + +case class LocalShuffleReaderExec(child: QueryStageExec) extends LeafExecNode { + + override def output: Seq[Attribute] = child.output + + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def outputPartitioning: Partitioning = { + + def tryReserveChildPartitioning(stage: ShuffleQueryStageExec): Partitioning = { + val initialPartitioning = stage.plan.child.outputPartitioning + if (initialPartitioning.isInstanceOf[UnknownPartitioning]) { + UnknownPartitioning(stage.plan.shuffleDependency.rdd.partitions.length) + } else { + initialPartitioning + } + } + + child match { + case stage: ShuffleQueryStageExec => + tryReserveChildPartitioning(stage) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + tryReserveChildPartitioning(stage) + } + } + + private var cachedShuffleRDD: RDD[InternalRow] = null + + override protected def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + stage.plan.createLocalShuffleRDD() + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + stage.plan.createLocalShuffleRDD() + } + } + cachedShuffleRDD + } + + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + append: String => Unit, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false, + maxFields: Int, + printNodeId: Boolean): Unit = { + super.generateTreeString(depth, + lastChildren, + append, + verbose, + prefix, + addSuffix, + maxFields, + printNodeId) + child.generateTreeString( + depth + 1, lastChildren :+ true, append, verbose, "", false, maxFields, printNodeId) + } +} 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 b072a7f5d914c..ca0cfb6834f1c 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import java.util.concurrent.TimeUnit._ +import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration @@ -171,6 +172,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) // This is very perf sensitive. // TODO: revisit this. We can consider reordering predicates as well. val generatedIsNotNullChecks = new Array[Boolean](notNullPreds.length) + val extraIsNotNullAttrs = mutable.Set[Attribute]() val generated = otherPreds.map { c => val nullChecks = c.references.map { r => val idx = notNullPreds.indexWhere { n => n.asInstanceOf[IsNotNull].child.semanticEquals(r)} @@ -178,6 +180,9 @@ case class FilterExec(condition: Expression, child: SparkPlan) generatedIsNotNullChecks(idx) = true // Use the child's output. The nullability is what the child produced. genPredicate(notNullPreds(idx), input, child.output) + } else if (notNullAttributes.contains(r.exprId) && !extraIsNotNullAttrs.contains(r)) { + extraIsNotNullAttrs += r + genPredicate(IsNotNull(r), input, child.output) } else { "" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index 85c36b7da9498..a43dc0b41ecca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.types._ private[columnar] trait ColumnAccessor { initialize() - protected def initialize() + protected def initialize(): Unit def hasNext: Boolean @@ -50,7 +50,7 @@ private[columnar] abstract class BasicColumnAccessor[JvmType]( protected val columnType: ColumnType[JvmType]) extends ColumnAccessor { - protected def initialize() {} + protected def initialize(): Unit = {} override def hasNext: Boolean = buffer.hasRemaining diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 542a10fc175c0..059e1447e31d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -221,7 +221,8 @@ private[columnar] object INT extends NativeColumnType(IntegerType, 4) { override def getField(row: InternalRow, ordinal: Int): Int = row.getInt(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setInt(toOrdinal, from.getInt(fromOrdinal)) } } @@ -249,7 +250,8 @@ private[columnar] object LONG extends NativeColumnType(LongType, 8) { override def getField(row: InternalRow, ordinal: Int): Long = row.getLong(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setLong(toOrdinal, from.getLong(fromOrdinal)) } } @@ -277,7 +279,8 @@ private[columnar] object FLOAT extends NativeColumnType(FloatType, 4) { override def getField(row: InternalRow, ordinal: Int): Float = row.getFloat(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setFloat(toOrdinal, from.getFloat(fromOrdinal)) } } @@ -305,7 +308,8 @@ private[columnar] object DOUBLE extends NativeColumnType(DoubleType, 8) { override def getField(row: InternalRow, ordinal: Int): Double = row.getDouble(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setDouble(toOrdinal, from.getDouble(fromOrdinal)) } } @@ -331,7 +335,8 @@ private[columnar] object BOOLEAN extends NativeColumnType(BooleanType, 1) { override def getField(row: InternalRow, ordinal: Int): Boolean = row.getBoolean(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setBoolean(toOrdinal, from.getBoolean(fromOrdinal)) } } @@ -359,7 +364,8 @@ private[columnar] object BYTE extends NativeColumnType(ByteType, 1) { override def getField(row: InternalRow, ordinal: Int): Byte = row.getByte(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setByte(toOrdinal, from.getByte(fromOrdinal)) } } @@ -387,7 +393,8 @@ private[columnar] object SHORT extends NativeColumnType(ShortType, 2) { override def getField(row: InternalRow, ordinal: Int): Short = row.getShort(ordinal) - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { to.setShort(toOrdinal, from.getShort(fromOrdinal)) } } @@ -452,7 +459,8 @@ private[columnar] object STRING row.getUTF8String(ordinal) } - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { setField(to, toOrdinal, getField(from, fromOrdinal)) } @@ -496,7 +504,8 @@ private[columnar] case class COMPACT_DECIMAL(precision: Int, scale: Int) row.setDecimal(ordinal, value, precision) } - override def copyField(from: InternalRow, fromOrdinal: Int, to: InternalRow, toOrdinal: Int) { + override def copyField(from: InternalRow, fromOrdinal: Int, + to: InternalRow, toOrdinal: Int): Unit = { setField(to, toOrdinal, getField(from, fromOrdinal)) } } 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 f7d4fa4c4ffcb..bb641bcb430ea 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.command import java.util.UUID import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException @@ -156,7 +157,11 @@ case class ExplainCommand( sparkSession.sessionState.executePlan(logicalPlan)) val outputString = if (codegen) { - codegenString(queryExecution.executedPlan) + try { + codegenString(queryExecution.executedPlan) + } catch { + case e: AnalysisException => e.toString + } } else if (extended) { queryExecution.toString } else if (cost) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala deleted file mode 100644 index 470c736da98b7..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.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.sql.execution.command - -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.types.StringType - - -/** - * A command for users to list the databases/schemas. - * If a databasePattern is supplied then the databases that only match the - * pattern would be listed. - * The syntax of using this command in SQL is: - * {{{ - * SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards']; - * }}} - */ -case class ShowDatabasesCommand(databasePattern: Option[String]) extends RunnableCommand { - - // The result of SHOW DATABASES has one column called 'databaseName' - override val output: Seq[Attribute] = { - AttributeReference("databaseName", StringType, nullable = false)() :: Nil - } - - override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - val databases = - databasePattern.map(catalog.listDatabases).getOrElse(catalog.listDatabases()) - databases.map { d => Row(d) } - } -} - - -/** - * Command for setting the current database. - * {{{ - * USE database_name; - * }}} - */ -case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { - - override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.catalog.setCurrentDatabase(databaseName) - 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 ee5d37cebf2f3..3645d38b3b558 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 @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit._ import scala.collection.{GenMap, GenSeq} import scala.collection.parallel.ForkJoinTaskSupport +import scala.collection.parallel.immutable.ParVector import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration @@ -132,6 +133,27 @@ case class AlterDatabasePropertiesCommand( } } +/** + * A command for users to set new location path for a database + * If the database does not exist, an error message will be issued to indicate the database + * does not exist. + * The syntax of using this command in SQL is: + * {{{ + * ALTER (DATABASE|SCHEMA) database_name SET LOCATION path + * }}} + */ +case class AlterDatabaseSetLocationCommand(databaseName: String, location: String) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + val oldDb = catalog.getDatabaseMetadata(databaseName) + catalog.alterDatabase(oldDb.copy(locationUri = CatalogUtils.stringToURI(location))) + + Seq.empty[Row] + } +} + /** * A command for users to show the name of the database, its comment (if one has been set), and its * root location on the filesystem. When extended is true, it also shows the database's properties @@ -663,7 +685,7 @@ case class AlterTableRecoverPartitionsCommand( val statusPar: GenSeq[FileStatus] = if (partitionNames.length > 1 && statuses.length > threshold || partitionNames.length > 2) { // parallelize the list of partitions here, then we can have better parallelism later. - val parArray = statuses.par + val parArray = new ParVector(statuses.toVector) parArray.tasksupport = evalTaskSupport parArray } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index d3b2491cd7056..6fdc7f4a58195 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionException} import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, FunctionResource} import org.apache.spark.sql.catalyst.expressions.{Attribute, ExpressionInfo} +import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -222,6 +223,21 @@ case class ShowFunctionsCommand( case (f, "USER") if showUserFunctions => f.unquotedString case (f, "SYSTEM") if showSystemFunctions => f.unquotedString } - functionNames.sorted.map(Row(_)) + // Hard code "<>", "!=", "between", and "case" for now as there is no corresponding functions. + // "<>", "!=", "between", and "case" is SystemFunctions, only show when showSystemFunctions=true + if (showSystemFunctions) { + (functionNames ++ + StringUtils.filterPattern(FunctionsCommand.virtualOperators, pattern.getOrElse("*"))) + .sorted.map(Row(_)) + } else { + functionNames.sorted.map(Row(_)) + } + } } + +object FunctionsCommand { + // operators that do not have corresponding functions. + // They should be handled `DescribeFunctionCommand`, `ShowFunctionsCommand` + val virtualOperators = Seq("!=", "<>", "between", "case") +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 9377cb0174673..f9992a925261f 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 @@ -278,6 +278,13 @@ case class LoadDataCommand( val catalog = sparkSession.sessionState.catalog val targetTable = catalog.getTableMetadata(table) val tableIdentwithDB = targetTable.identifier.quotedString + val normalizedSpec = partition.map { spec => + PartitioningUtils.normalizePartitionSpec( + spec, + targetTable.partitionColumnNames, + tableIdentwithDB, + sparkSession.sessionState.conf.resolver) + } if (targetTable.tableType == CatalogTableType.VIEW) { throw new AnalysisException(s"Target table in LOAD DATA cannot be a view: $tableIdentwithDB") @@ -297,13 +304,6 @@ case class LoadDataCommand( s"do not match number of partitioned columns in table " + s"(${targetTable.partitionColumnNames.size})") } - partition.get.keys.foreach { colName => - if (!targetTable.partitionColumnNames.contains(colName)) { - throw new AnalysisException(s"LOAD DATA target table $tableIdentwithDB is partitioned, " + - s"but the specified partition spec refers to a column that is not partitioned: " + - s"'$colName'") - } - } } else { if (partition.nonEmpty) { throw new AnalysisException(s"LOAD DATA target table $tableIdentwithDB is not " + @@ -353,7 +353,7 @@ case class LoadDataCommand( catalog.loadPartition( targetTable.identifier, loadPath.toString, - partition.get, + normalizedSpec.get, isOverwrite, inheritTableSpecs = true, isSrcLocal = isLocal) 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 0f5f1591623af..e9b8fae7cd735 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 @@ -378,8 +378,6 @@ case class DataSource( // This is a non-streaming file based datasource. case (format: FileFormat, _) => - val globbedPaths = - checkAndGlobPathIfNecessary(checkEmptyGlobPath = true, checkFilesExist = checkFilesExist) val useCatalogFileIndex = sparkSession.sqlContext.conf.manageFilesourcePartitions && catalogTable.isDefined && catalogTable.get.tracksPartitionsInCatalog && catalogTable.get.partitionColumnNames.nonEmpty @@ -391,6 +389,8 @@ case class DataSource( catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize)) (index, catalogTable.get.dataSchema, catalogTable.get.partitionSchema) } else { + val globbedPaths = checkAndGlobPathIfNecessary( + checkEmptyGlobPath = true, checkFilesExist = checkFilesExist) val index = createInMemoryFileIndex(globbedPaths) val (resultDataSchema, resultPartitionSchema) = getOrInferFileFormatSchema(format, () => index) 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 deleted file mode 100644 index b9b8e069d5237..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ /dev/null @@ -1,393 +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 scala.collection.mutable - -import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, LookupCatalog, TableCatalog} -import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} -import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} - -case class DataSourceResolution( - conf: SQLConf, - catalogManager: CatalogManager) - extends Rule[LogicalPlan] with CastSupport with LookupCatalog { - - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case CreateTableStatement( - AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, - V1Provider(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(sessionCatalog.asTableCatalog, identifier, create) - } - - case CreateTableAsSelectStatement( - AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, - V1Provider(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(sessionCatalog.asTableCatalog, identifier, create) - } - - case DescribeColumnStatement( - AsTableIdentifier(tableName), colName, isExtended) => - DescribeColumnCommand(tableName, colName, isExtended) - - case DescribeColumnStatement( - CatalogObjectIdentifier(Some(catalog), ident), colName, isExtended) => - throw new AnalysisException("Describing columns is not supported for v2 tables.") - - case DescribeTableStatement( - AsTableIdentifier(tableName), partitionSpec, isExtended) => - DescribeTableCommand(tableName, partitionSpec, isExtended) - - case ReplaceTableStatement( - AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, - V1Provider(provider), options, location, comment, orCreate) => - throw new AnalysisException( - s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + - s" API. Write provider name: $provider, identifier: $table.") - - case ReplaceTableAsSelectStatement( - AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, - V1Provider(provider), options, location, comment, orCreate) => - throw new AnalysisException( - s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + - s" API. Write provider name: $provider, identifier: $table.") - - case replace: ReplaceTableStatement => - // the provider was not a v1 source, convert to a v2 plan - val CatalogObjectIdentifier(maybeCatalog, identifier) = replace.tableName - val catalog = maybeCatalog.getOrElse(sessionCatalog).asTableCatalog - convertReplaceTable(catalog, identifier, replace) - - case rtas: ReplaceTableAsSelectStatement => - // the provider was not a v1 source, convert to a v2 plan - val CatalogObjectIdentifier(maybeCatalog, identifier) = rtas.tableName - val catalog = maybeCatalog.getOrElse(sessionCatalog).asTableCatalog - convertRTAS(catalog, identifier, rtas) - - 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 DeleteFromStatement(AsTableIdentifier(table), tableAlias, condition) => - throw new AnalysisException( - s"Delete from tables is not supported using the legacy / v1 Spark external catalog" + - s" API. Identifier: $table.") - - case delete: DeleteFromStatement => - val relation = UnresolvedRelation(delete.tableName) - val aliased = delete.tableAlias.map(SubqueryAlias(_, relation)).getOrElse(relation) - DeleteFromTable(aliased, delete.condition) - - case ShowNamespacesStatement(None, pattern) => - defaultCatalog match { - case Some(catalog) => - ShowNamespaces(catalog.asNamespaceCatalog, None, pattern) - case None => - throw new AnalysisException("No default v2 catalog is set.") - } - - case ShowNamespacesStatement(Some(namespace), pattern) => - val CatalogNamespace(maybeCatalog, ns) = namespace - maybeCatalog match { - case Some(catalog) => - ShowNamespaces(catalog.asNamespaceCatalog, Some(ns), pattern) - case None => - throw new AnalysisException( - s"No v2 catalog is available for ${namespace.quoted}") - } - - case ShowTablesStatement(None, pattern) => - defaultCatalog match { - case Some(catalog) => - ShowTables( - catalog.asTableCatalog, - catalogManager.currentNamespace, - pattern) - case None => - ShowTablesCommand(None, pattern) - } - - case ShowTablesStatement(Some(namespace), pattern) => - val CatalogNamespace(maybeCatalog, ns) = namespace - maybeCatalog match { - case Some(catalog) => - ShowTables(catalog.asTableCatalog, ns, pattern) - case None => - if (namespace.length != 1) { - throw new AnalysisException( - s"The database name is not valid: ${namespace.quoted}") - } - ShowTablesCommand(Some(namespace.quoted), pattern) - } - } - - object V1Provider { - def unapply(provider: String): Option[String] = { - DataSource.lookupDataSourceV2(provider, conf) match { - // TODO(SPARK-28396): Currently file source v2 can't work with tables. - case Some(_: FileDataSourceV2) => Some(provider) - case Some(_) => 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 convertRTAS( - catalog: TableCatalog, - identifier: Identifier, - rtas: ReplaceTableAsSelectStatement): ReplaceTableAsSelect = { - // convert the bucket spec and add it as a transform - val partitioning = rtas.partitioning ++ rtas.bucketSpec.map(_.asTransform) - val properties = convertTableProperties( - rtas.properties, rtas.options, rtas.location, rtas.comment, rtas.provider) - - ReplaceTableAsSelect( - catalog, - identifier, - partitioning, - rtas.asSelect, - properties, - writeOptions = rtas.options.filterKeys(_ != "path"), - orCreate = rtas.orCreate) - } - - private def convertReplaceTable( - catalog: TableCatalog, - identifier: Identifier, - replace: ReplaceTableStatement): ReplaceTable = { - // convert the bucket spec and add it as a transform - val partitioning = replace.partitioning ++ replace.bucketSpec.map(_.asTransform) - val properties = convertTableProperties( - replace.properties, replace.options, replace.location, replace.comment, replace.provider) - - ReplaceTable( - catalog, - identifier, - replace.tableSchema, - partitioning, - properties, - orCreate = replace.orCreate) - } - - 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 1a92a2bb7156c..211642d78aabe 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 @@ -34,8 +34,7 @@ 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, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ 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 index 70e78a9ecc859..5f6c3e8f7eeed 100644 --- 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 @@ -20,8 +20,7 @@ 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.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2, FileTable} 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 cf7a13050f66c..71d61349fcc38 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 @@ -121,7 +121,7 @@ class InMemoryFileIndex( case None => pathsToFetch += path } - Unit // for some reasons scalac 2.12 needs this; return type doesn't matter + () // for some reasons scalac 2.12 needs this; return type doesn't matter } val filter = FileInputFormat.getInputPathFilter(new JobConf(hadoopConf, this.getClass)) val discovered = InMemoryFileIndex.bulkListLeafFiles( @@ -171,6 +171,7 @@ object InMemoryFileIndex extends Logging { areRootPaths: Boolean): Seq[(Path, Seq[FileStatus])] = { val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles + val ignoreLocality = sparkSession.sessionState.conf.ignoreDataLocality // Short-circuits parallel listing when serial listing is likely to be faster. if (paths.size <= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { @@ -181,6 +182,7 @@ object InMemoryFileIndex extends Logging { filter, Some(sparkSession), ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, isRootPath = areRootPaths) (path, leafFiles) } @@ -221,6 +223,7 @@ object InMemoryFileIndex extends Logging { filter, None, ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, isRootPath = areRootPaths) (path, leafFiles) }.iterator @@ -287,6 +290,7 @@ object InMemoryFileIndex extends Logging { filter: PathFilter, sessionOpt: Option[SparkSession], ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, isRootPath: Boolean): Seq[FileStatus] = { logTrace(s"Listing $path") val fs = path.getFileSystem(hadoopConf) @@ -299,7 +303,7 @@ object InMemoryFileIndex extends Logging { // to retrieve the file status with the file block location. The reason to still fallback // to listStatus is because the default implementation would potentially throw a // FileNotFoundException which is better handled by doing the lookups manually below. - case _: DistributedFileSystem => + case _: DistributedFileSystem if !ignoreLocality => val remoteIter = fs.listLocatedStatus(path) new Iterator[LocatedFileStatus]() { def next(): LocatedFileStatus = remoteIter.next @@ -353,6 +357,7 @@ object InMemoryFileIndex extends Logging { filter, sessionOpt, ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, isRootPath = false) } } @@ -376,7 +381,7 @@ object InMemoryFileIndex extends Logging { // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not // be a big deal since we always use to `bulkListLeafFiles` when the number of // paths exceeds threshold. - case f => + case f if !ignoreLocality => // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), // which is very slow on some file system (RawLocalFileSystem, which is launch a // subprocess and parse the stdout). @@ -400,6 +405,8 @@ object InMemoryFileIndex extends Logging { missingFiles += f.getPath.toString None } + + case f => Some(f) } if (missingFiles.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index d43fa3893df1d..fbe874b3e8bc5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -90,8 +90,6 @@ case class InsertIntoHadoopFsRelationCommand( fs, catalogTable.get, qualifiedOutputPath, matchingPartitions) } - val pathExists = fs.exists(qualifiedOutputPath) - val parameters = CaseInsensitiveMap(options) val partitionOverwriteMode = parameters.get("partitionOverwriteMode") @@ -111,25 +109,30 @@ case class InsertIntoHadoopFsRelationCommand( outputPath = outputPath.toString, dynamicPartitionOverwrite = dynamicPartitionOverwrite) - val doInsertion = (mode, pathExists) match { - case (SaveMode.ErrorIfExists, true) => - throw new AnalysisException(s"path $qualifiedOutputPath already exists.") - case (SaveMode.Overwrite, true) => - if (ifPartitionNotExists && matchingPartitions.nonEmpty) { - false - } else if (dynamicPartitionOverwrite) { - // For dynamic partition overwrite, do not delete partition directories ahead. - true - } else { - deleteMatchingPartitions(fs, qualifiedOutputPath, customPartitionLocations, committer) + val doInsertion = if (mode == SaveMode.Append) { + true + } else { + val pathExists = fs.exists(qualifiedOutputPath) + (mode, pathExists) match { + case (SaveMode.ErrorIfExists, true) => + throw new AnalysisException(s"path $qualifiedOutputPath already exists.") + case (SaveMode.Overwrite, true) => + if (ifPartitionNotExists && matchingPartitions.nonEmpty) { + false + } else if (dynamicPartitionOverwrite) { + // For dynamic partition overwrite, do not delete partition directories ahead. + true + } else { + deleteMatchingPartitions(fs, qualifiedOutputPath, customPartitionLocations, committer) + true + } + case (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => true - } - case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => - true - case (SaveMode.Ignore, exists) => - !exists - case (s, exists) => - throw new IllegalStateException(s"unsupported save mode $s ($exists)") + case (SaveMode.Ignore, exists) => + !exists + case (s, exists) => + throw new IllegalStateException(s"unsupported save mode $s ($exists)") + } } if (doInsertion) { 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 d184f3cb71b1a..5d1feaed81a9a 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 @@ -147,14 +147,7 @@ class JDBCOptions( """.stripMargin ) - val fetchSize = { - val size = parameters.getOrElse(JDBC_BATCH_FETCH_SIZE, "0").toInt - require(size >= 0, - s"Invalid value `${size.toString}` for parameter " + - s"`$JDBC_BATCH_FETCH_SIZE`. The minimum value is 0. When the value is 0, " + - "the JDBC driver ignores the value and does the estimates.") - size - } + val fetchSize = parameters.getOrElse(JDBC_BATCH_FETCH_SIZE, "0").toInt // ------------------------------------------------------------ // Optional parameters only for writing diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 16b493892e3be..e25ce53941ff6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -231,7 +231,7 @@ private[jdbc] class JDBCRDD( var stmt: PreparedStatement = null var conn: Connection = null - def close() { + def close(): Unit = { if (closed) return try { if (null != rs) { 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 815b62dfbf898..fb5daa4bfae3a 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 @@ -328,32 +328,28 @@ class ParquetFileFormat iter.asInstanceOf[Iterator[InternalRow]] } else { logDebug(s"Falling back to parquet-mr") - // ParquetRecordReader returns UnsafeRow + // ParquetRecordReader returns InternalRow 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) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](readSupport) + new ParquetRecordReader[InternalRow](readSupport) } - val iter = new RecordReaderIterator(reader) + val iter = new RecordReaderIterator[InternalRow](reader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - // This is a horrible erasure hack... if we type the iterator above, then it actually check - // the type in next() and we get a class cast exception. If we make that function return - // Object, then we can defer the cast until later! if (partitionSchema.length == 0) { // There is no partition columns - iter.asInstanceOf[Iterator[InternalRow]] + iter.map(unsafeProjection) } else { - iter.asInstanceOf[Iterator[InternalRow]] - .map(d => appendPartitionColumns(joinedRow(d, file.partitionValues))) + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) } } } 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 2c7231d2c3e0a..69c8bad5f1c83 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 @@ -29,13 +29,13 @@ import org.apache.parquet.schema._ import org.apache.parquet.schema.Type.Repetition import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** * A Parquet [[ReadSupport]] implementation for reading Parquet records as Catalyst - * [[UnsafeRow]]s. + * [[InternalRow]]s. * * The API interface of [[ReadSupport]] is a little bit over complicated because of historical * reasons. In older versions of parquet-mr (say 1.6.0rc3 and prior), [[ReadSupport]] need to be @@ -51,7 +51,7 @@ import org.apache.spark.sql.types._ */ class ParquetReadSupport(val convertTz: Option[TimeZone], enableVectorizedReader: Boolean) - extends ReadSupport[UnsafeRow] with Logging { + extends ReadSupport[InternalRow] with Logging { private var catalystRequestedSchema: StructType = _ def this() { @@ -114,13 +114,13 @@ class ParquetReadSupport(val convertTz: Option[TimeZone], /** * Called on executor side after [[init()]], before instantiating actual Parquet record readers. * Responsible for instantiating [[RecordMaterializer]], which is used for converting Parquet - * records to Catalyst [[UnsafeRow]]s. + * records to Catalyst [[InternalRow]]s. */ override def prepareForRead( conf: Configuration, keyValueMetaData: JMap[String, String], fileSchema: MessageType, - readContext: ReadContext): RecordMaterializer[UnsafeRow] = { + readContext: ReadContext): RecordMaterializer[InternalRow] = { val parquetRequestedSchema = readContext.getRequestedSchema new ParquetRecordMaterializer( parquetRequestedSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala index b2459dd0e8bba..3098a332d3027 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala @@ -22,7 +22,7 @@ import java.util.TimeZone import org.apache.parquet.io.api.{GroupConverter, RecordMaterializer} import org.apache.parquet.schema.MessageType -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.StructType /** @@ -37,12 +37,12 @@ private[parquet] class ParquetRecordMaterializer( catalystSchema: StructType, schemaConverter: ParquetToSparkSchemaConverter, convertTz: Option[TimeZone]) - extends RecordMaterializer[UnsafeRow] { + extends RecordMaterializer[InternalRow] { private val rootConverter = new ParquetRowConverter(schemaConverter, parquetSchema, catalystSchema, convertTz, NoopUpdater) - override def getCurrentRecord: UnsafeRow = rootConverter.currentRecord + override def getCurrentRecord: InternalRow = rootConverter.currentRecord override def getRootConverter: GroupConverter = rootConverter } 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 b772b6b77d1ce..ff5c724375c33 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 @@ -173,12 +173,10 @@ private[parquet] class ParquetRowConverter( private val currentRow = new SpecificInternalRow(catalystType.map(_.dataType)) - private val unsafeProjection = UnsafeProjection.create(catalystType) - /** - * The [[UnsafeRow]] converted from an entire Parquet record. + * The [[InternalRow]] converted from an entire Parquet record. */ - def currentRecord: UnsafeRow = unsafeProjection(currentRow) + def currentRecord: InternalRow = currentRow // Converters for each field. private val fieldConverters: Array[Converter with HasParentContainerUpdater] = { 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 c92c68095db8f..95343e2872def 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 @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.execution.command.DDLUtils @@ -190,14 +189,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi query } - // SPARK-28730: for V1 data source, we use the "LEGACY" as default store assignment policy. - // TODO: use ANSI store assignment policy by default in SPARK-28495. - val storeAssignmentPolicy = conf.storeAssignmentPolicy.getOrElse(StoreAssignmentPolicy.LEGACY) c.copy( tableDesc = existingTable, query = Some(TableOutputResolver.resolveOutputColumns( tableDesc.qualifiedName, existingTable.schema.toAttributes, newQuery, - byName = true, conf, storeAssignmentPolicy))) + byName = true, conf))) // Here we normalize partition, bucket and sort column names, w.r.t. the case sensitivity // config, and do various checks: @@ -403,11 +399,8 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { s"including ${staticPartCols.size} partition column(s) having constant value(s).") } - // SPARK-28730: for V1 data source, we use the "LEGACY" as default store assignment policy. - // TODO: use ANSI store assignment policy by default in SPARK-28495. - val storeAssignmentPolicy = conf.storeAssignmentPolicy.getOrElse(StoreAssignmentPolicy.LEGACY) val newQuery = TableOutputResolver.resolveOutputColumns( - tblName, expectedColumns, insert.query, byName = false, conf, storeAssignmentPolicy) + tblName, expectedColumns, insert.query, byName = false, conf) if (normalizedPartSpec.nonEmpty) { if (normalizedPartSpec.size != partColNames.length) { throw new AnalysisException( 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 index 4525f2608b710..8b2930cca841d 100644 --- 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 @@ -18,11 +18,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.SparkException -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.connector.catalog.{Identifier, TableCatalog, TableChange} -import org.apache.spark.sql.execution.LeafExecNode /** * Physical plan node for altering a table. @@ -30,11 +28,11 @@ import org.apache.spark.sql.execution.LeafExecNode case class AlterTableExec( catalog: TableCatalog, ident: Identifier, - changes: Seq[TableChange]) extends LeafExecNode { + changes: Seq[TableChange]) extends V2CommandExec { override def output: Seq[Attribute] = Seq.empty - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { try { catalog.alterTable(ident, changes: _*) } catch { @@ -42,6 +40,6 @@ case class AlterTableExec( throw new SparkException(s"Unsupported table change: ${e.getMessage}", e) } - sqlContext.sparkContext.parallelize(Seq.empty, 1) + Seq.empty } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala new file mode 100644 index 0000000000000..0f69f85dd8376 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.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.execution.datasources.v2 + +import scala.collection.JavaConverters.mapAsJavaMapConverter + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.SupportsNamespaces + +/** + * Physical plan node for creating a namespace. + */ +case class CreateNamespaceExec( + catalog: SupportsNamespaces, + namespace: Seq[String], + ifNotExists: Boolean, + private var properties: Map[String, String]) + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val ns = namespace.toArray + if (!catalog.namespaceExists(ns)) { + try { + catalog.createNamespace(ns, properties.asJava) + } catch { + case _: NamespaceAlreadyExistsException if ifNotExists => + logWarning(s"Namespace ${namespace.quoted} was created concurrently. Ignoring.") + } + } else if (!ifNotExists) { + throw new NamespaceAlreadyExistsException(ns) + } + + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} 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 index 600ed6f7c98ef..511cd8a9a438f 100644 --- 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 @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ -import org.apache.spark.rdd.RDD 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.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType case class CreateTableExec( @@ -34,10 +32,10 @@ case class CreateTableExec( tableSchema: StructType, partitioning: Seq[Transform], tableProperties: Map[String, String], - ignoreIfExists: Boolean) extends LeafExecNode { + ignoreIfExists: Boolean) extends V2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { if (!catalog.tableExists(identifier)) { try { catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) @@ -49,7 +47,7 @@ case class CreateTableExec( throw new TableAlreadyExistsException(identifier) } - sqlContext.sparkContext.parallelize(Seq.empty, 1) + Seq.empty } override def output: Seq[Attribute] = Seq.empty 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 6387333d06880..4a7cb7db45ded 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 @@ -23,7 +23,7 @@ import scala.collection.mutable 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.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} @@ -193,6 +193,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { catalog, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil } + case RefreshTable(catalog, ident) => + RefreshTableExec(catalog, ident) :: Nil + case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => catalog match { case staging: StagingTableCatalog => @@ -289,12 +292,18 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case CreateNamespace(catalog, namespace, ifNotExists, properties) => + CreateNamespaceExec(catalog, namespace, ifNotExists, properties) :: Nil + case r: ShowNamespaces => ShowNamespacesExec(r.output, r.catalog, r.namespace, r.pattern) :: Nil case r : ShowTables => ShowTablesExec(r.output, r.catalog, r.namespace, r.pattern) :: Nil + case SetCatalogAndNamespace(catalogManager, catalogName, namespace) => + SetCatalogAndNamespaceExec(catalogManager, catalogName, namespace) :: Nil + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala index b91cbca8d2e70..afebbfd01db22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala @@ -17,21 +17,18 @@ 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.connector.catalog.SupportsDelete -import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.util.CaseInsensitiveStringMap case class DeleteFromTableExec( table: SupportsDelete, - condition: Array[Filter]) extends LeafExecNode { + condition: Array[Filter]) extends V2CommandExec { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { table.deleteWhere(condition) - sparkContext.emptyRDD + Seq.empty } override def output: Seq[Attribute] = Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index e696efe950f98..a032f5c9d5684 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -20,22 +20,20 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.connector.catalog.Table -import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType case class DescribeTableExec( output: Seq[Attribute], table: Table, - isExtended: Boolean) extends LeafExecNode { + isExtended: Boolean) extends V2CommandExec { private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) @@ -43,7 +41,7 @@ case class DescribeTableExec( addPartitioning(rows) addProperties(rows) } - sparkContext.parallelize(rows) + rows } private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { 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 index 9164b211ed0e5..967613f77577c 100644 --- 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 @@ -17,27 +17,25 @@ 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.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} -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 { + extends V2CommandExec { - override def doExecute(): RDD[InternalRow] = { + override def run(): Seq[InternalRow] = { if (catalog.tableExists(ident)) { catalog.dropTable(ident) } else if (!ifExists) { throw new NoSuchTableException(ident) } - sqlContext.sparkContext.parallelize(Seq.empty, 1) + Seq.empty } override def output: Seq[Attribute] = Seq.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala similarity index 70% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala index d41e8a5010257..2a19ff304a9e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala @@ -15,20 +15,19 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.plans.logical.sql +package org.apache.spark.sql.execution.datasources.v2 +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} -/** - * A DROP TABLE statement, as parsed from SQL. - */ -case class DropTableStatement( - tableName: Seq[String], - ifExists: Boolean, - purge: Boolean) extends ParsedStatement { +case class RefreshTableExec( + catalog: TableCatalog, + ident: Identifier) extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + catalog.invalidateTable(ident) + Seq.empty + } override def output: Seq[Attribute] = Seq.empty - - override def children: Seq[LogicalPlan] = Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index a56ccd6a6a9f5..1f3bcf2e3fe57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -35,16 +33,16 @@ case class ReplaceTableExec( tableSchema: StructType, partitioning: Seq[Transform], tableProperties: Map[String, String], - orCreate: Boolean) extends LeafExecNode { + orCreate: Boolean) extends V2CommandExec { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { if (catalog.tableExists(ident)) { catalog.dropTable(ident) } else if (!orCreate) { throw new CannotReplaceMissingTableException(ident) } catalog.createTable(ident, tableSchema, partitioning.toArray, tableProperties.asJava) - sqlContext.sparkContext.parallelize(Seq.empty, 1) + Seq.empty } override def output: Seq[Attribute] = Seq.empty @@ -56,9 +54,9 @@ case class AtomicReplaceTableExec( tableSchema: StructType, partitioning: Seq[Transform], tableProperties: Map[String, String], - orCreate: Boolean) extends LeafExecNode { + orCreate: Boolean) extends V2CommandExec { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { val staged = if (orCreate) { catalog.stageCreateOrReplace( identifier, tableSchema, partitioning.toArray, tableProperties.asJava) @@ -74,8 +72,7 @@ case class AtomicReplaceTableExec( throw new CannotReplaceMissingTableException(identifier) } commitOrAbortStagedChanges(staged) - - sqlContext.sparkContext.parallelize(Seq.empty, 1) + Seq.empty } override def output: Seq[Attribute] = Seq.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala similarity index 55% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala index 523158788e834..9e6f00e0923ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala @@ -15,19 +15,28 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.plans.logical.sql +package org.apache.spark.sql.execution.datasources.v2 +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.CatalogManager /** - * A DROP VIEW statement, as parsed from SQL. + * Physical plan node for setting the current catalog and/or namespace. */ -case class DropViewStatement( - viewName: Seq[String], - ifExists: Boolean) extends ParsedStatement { +case class SetCatalogAndNamespaceExec( + catalogManager: CatalogManager, + catalogName: Option[String], + namespace: Option[Seq[String]]) + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + // The catalog is updated first because CatalogManager resets the current namespace + // when the current catalog is set. + catalogName.map(catalogManager.setCurrentCatalog) + namespace.map(ns => catalogManager.setCurrentNamespace(ns.toArray)) - override def output: Seq[Attribute] = Seq.empty + Seq.empty + } - override def children: Seq[LogicalPlan] = Seq.empty + override def output: Seq[Attribute] = Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala index 49f55be52de16..15402cc0074ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.connector.catalog.SupportsNamespaces -import org.apache.spark.sql.execution.LeafExecNode /** * Physical plan node for showing namespaces. @@ -36,9 +34,9 @@ case class ShowNamespacesExec( catalog: SupportsNamespaces, namespace: Option[Seq[String]], pattern: Option[String]) - extends LeafExecNode { + extends V2CommandExec { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { val namespaces = namespace.map { ns => if (ns.nonEmpty) { catalog.listNamespaces(ns.toArray) @@ -59,6 +57,6 @@ case class ShowNamespacesExec( } } - sparkContext.parallelize(rows, 1) + rows } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index e352bc6759623..995b00871fc2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.connector.catalog.TableCatalog -import org.apache.spark.sql.execution.LeafExecNode /** * Physical plan node for showing tables. @@ -36,8 +34,8 @@ case class ShowTablesExec( catalog: TableCatalog, namespace: Seq[String], pattern: Option[String]) - extends LeafExecNode { - override protected def doExecute(): RDD[InternalRow] = { + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() val encoder = RowEncoder(schema).resolveAndBind() @@ -53,6 +51,6 @@ case class ShowTablesExec( } } - sparkContext.parallelize(rows, 1) + rows } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala new file mode 100644 index 0000000000000..6ee60e9c4512f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.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.execution.LeafExecNode + +/** + * A physical operator that executes run() and saves the result to prevent multiple executions. + * Any V2 commands that do not require triggering a spark job should extend this class. + */ +abstract class V2CommandExec extends LeafExecNode { + + /** + * Abstract method that each concrete command needs to implement to compute the result. + */ + protected def run(): Seq[InternalRow] + + /** + * The value of this field can be used as the contents of the corresponding RDD generated from + * the physical plan of this command. + */ + private lazy val result: Seq[InternalRow] = run() + + /** + * The `execute()` method of all the physical command classes should reference `result` + * so that the command can be executed eagerly right after the command query is created. + */ + override def executeCollect(): Array[InternalRow] = result.toArray + + override def executeToIterator: Iterator[InternalRow] = result.toIterator + + override def executeTake(limit: Int): Array[InternalRow] = result.take(limit).toArray + + protected override def doExecute(): RDD[InternalRow] = { + sqlContext.sparkContext.parallelize(result, 1) + } +} 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 index 525d9c4e541e6..dffb9cb67b5c2 100644 --- 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 @@ -52,7 +52,10 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) 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 + catalog + .listTables(db) + .map(ident => Identifier.of(Array(ident.database.getOrElse("")), ident.table)) + .toArray case _ => throw new NoSuchNamespaceException(namespace) } 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 index ded145f7de94e..b2fc724057eba 100644 --- 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 @@ -31,7 +31,6 @@ 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.connector.read.{InputPartition, PartitionReader} import org.apache.spark.sql.execution.datasources.{PartitionedFile, RecordReaderIterator} @@ -176,7 +175,7 @@ case class ParquetPartitionReaderFactory( reader } - private def createRowBaseReader(file: PartitionedFile): RecordReader[Void, UnsafeRow] = { + private def createRowBaseReader(file: PartitionedFile): RecordReader[Void, InternalRow] = { buildReaderBase(file, createRowBaseParquetReader) } @@ -185,16 +184,16 @@ case class ParquetPartitionReaderFactory( partitionValues: InternalRow, hadoopAttemptContext: TaskAttemptContextImpl, pushed: Option[FilterPredicate], - convertTz: Option[TimeZone]): RecordReader[Void, UnsafeRow] = { + convertTz: Option[TimeZone]): RecordReader[Void, InternalRow] = { logDebug(s"Falling back to parquet-mr") val taskContext = Option(TaskContext.get()) - // ParquetRecordReader returns UnsafeRow + // ParquetRecordReader returns InternalRow 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) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](readSupport) + new ParquetRecordReader[InternalRow](readSupport) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion listener before `initialization`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala index 3315ae7dabef1..a1dde415d6e8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{ExplainUtils, LeafExecNode, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -109,9 +109,10 @@ case class ReuseExchange(conf: SQLConf) extends Rule[SparkPlan] { } // Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls. val exchanges = mutable.HashMap[StructType, ArrayBuffer[Exchange]]() - plan.transformUp { + + // Replace a Exchange duplicate with a ReusedExchange + def reuse: PartialFunction[Exchange, SparkPlan] = { case exchange: Exchange => - // the exchanges that have same results usually also have same schemas (same column names). val sameSchema = exchanges.getOrElseUpdate(exchange.schema, ArrayBuffer[Exchange]()) val samePlan = sameSchema.find { e => exchange.sameResult(e) @@ -125,5 +126,16 @@ case class ReuseExchange(conf: SQLConf) extends Rule[SparkPlan] { exchange } } + + plan transformUp { + case exchange: Exchange => reuse(exchange) + } transformAllExpressions { + // Lookup inside subqueries for duplicate exchanges + case in: InSubqueryExec => + val newIn = in.plan.transformUp { + case exchange: Exchange => reuse(exchange) + } + in.copy(plan = newIn.asInstanceOf[BaseSubqueryExec]) + } } } 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 2f4c5734469f8..2f94c522712b1 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 @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns 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.adaptive.LocalShuffledRowRDD 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 @@ -82,6 +83,10 @@ case class ShuffleExchangeExec( new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } + def createLocalShuffleRDD(): LocalShuffledRowRDD = { + new LocalShuffledRowRDD(shuffleDependency, readMetrics) + } + /** * Caches the created ShuffleRowRDD so we can reuse that. */ 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 189727a9bc88d..26fb0e5ffb1af 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 @@ -191,7 +191,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -235,7 +236,8 @@ case class SortMergeJoinExec( streamedIter = RowIterator.fromScala(leftIter), bufferedIter = RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) val rightNullRow = new GenericInternalRow(right.output.length) new LeftOuterIterator( @@ -249,7 +251,8 @@ case class SortMergeJoinExec( streamedIter = RowIterator.fromScala(rightIter), bufferedIter = RowIterator.fromScala(leftIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) val leftNullRow = new GenericInternalRow(left.output.length) new RightOuterIterator( @@ -283,7 +286,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -318,7 +322,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -360,7 +365,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -640,6 +646,9 @@ case class SortMergeJoinExec( (evaluateVariables(leftVars), "") } + val thisPlan = ctx.addReferenceObj("plan", this) + val eagerCleanup = s"$thisPlan.cleanupResources();" + s""" |while (findNextInnerJoinRows($leftInput, $rightInput)) { | ${leftVarDecl.mkString("\n")} @@ -653,6 +662,7 @@ case class SortMergeJoinExec( | } | if (shouldStop()) return; |} + |$eagerCleanup """.stripMargin } } @@ -678,6 +688,7 @@ case class SortMergeJoinExec( * @param inMemoryThreshold Threshold for number of rows guaranteed to be held in memory by * internal buffer * @param spillThreshold Threshold for number of rows to be spilled by internal buffer + * @param eagerCleanupResources the eager cleanup function to be invoked when no join row found */ private[joins] class SortMergeJoinScanner( streamedKeyGenerator: Projection, @@ -686,7 +697,8 @@ private[joins] class SortMergeJoinScanner( streamedIter: RowIterator, bufferedIter: RowIterator, inMemoryThreshold: Int, - spillThreshold: Int) { + spillThreshold: Int, + eagerCleanupResources: () => Unit) { private[this] var streamedRow: InternalRow = _ private[this] var streamedRowKey: InternalRow = _ private[this] var bufferedRow: InternalRow = _ @@ -710,7 +722,8 @@ private[joins] class SortMergeJoinScanner( def getBufferedMatches: ExternalAppendOnlyUnsafeRowArray = bufferedMatches /** - * Advances both input iterators, stopping when we have found rows with matching join keys. + * Advances both input iterators, stopping when we have found rows with matching join keys. If no + * join rows found, try to do the eager resources cleanup. * @return true if matching rows have been found and false otherwise. If this returns true, then * [[getStreamedRow]] and [[getBufferedMatches]] can be called to construct the join * results. @@ -720,7 +733,7 @@ private[joins] class SortMergeJoinScanner( // Advance the streamed side of the join until we find the next row whose join key contains // no nulls or we hit the end of the streamed iterator. } - if (streamedRow == null) { + val found = if (streamedRow == null) { // We have consumed the entire streamed iterator, so there can be no more matches. matchJoinKey = null bufferedMatches.clear() @@ -760,17 +773,19 @@ private[joins] class SortMergeJoinScanner( true } } + if (!found) eagerCleanupResources() + found } /** * Advances the streamed input iterator and buffers all rows from the buffered input that - * have matching keys. + * have matching keys. If no join rows found, try to do the eager resources cleanup. * @return true if the streamed iterator returned a row, false otherwise. If this returns true, * then [[getStreamedRow]] and [[getBufferedMatches]] can be called to produce the outer * join results. */ final def findNextOuterJoinRows(): Boolean = { - if (!advancedStreamed()) { + val found = if (!advancedStreamed()) { // We have consumed the entire streamed iterator, so there can be no more matches. matchJoinKey = null bufferedMatches.clear() @@ -800,6 +815,8 @@ private[joins] class SortMergeJoinScanner( // If there is a streamed input then we always return true true } + if (!found) eagerCleanupResources() + found } // --- Private methods -------------------------------------------------------------------------- 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 fcf68467460bb..b44b13c8de0da 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 @@ -30,6 +30,7 @@ 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.ColumnarBatch import org.apache.spark.util.Utils /** @@ -42,8 +43,8 @@ class ArrowPythonRunner( schema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BaseArrowPythonRunner[Iterator[InternalRow]]( - funcs, evalType, argOffsets) { + extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch](funcs, evalType, argOffsets) + with PythonArrowOutput { override val bufferSize: Int = SQLConf.get.pandasUDFBufferSize require( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala similarity index 82% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala index 8ea9881c575a1..25ce16db264ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala @@ -17,27 +17,27 @@ package org.apache.spark.sql.execution.python -import java.io._ -import java.net._ +import java.io.DataOutputStream +import java.net.Socket import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.ArrowStreamWriter -import org.apache.spark._ -import org.apache.spark.api.python._ +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, PythonRDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.arrow.ArrowWriter -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils /** - * Python UDF Runner for cogrouped udfs. Although the data is exchanged with the python - * worker via arrow, we cannot use `ArrowPythonRunner` as we need to send more than one - * dataframe. + * Python UDF Runner for cogrouped udfs. It sends Arrow bathes from two different DataFrames, + * groups them in Python, and receive it back in JVM as batches of single DataFrame. */ -class CogroupedArrowPythonRunner( +class CoGroupedArrowPythonRunner( funcs: Seq[ChainedPythonFunctions], evalType: Int, argOffsets: Array[Array[Int]], @@ -45,8 +45,9 @@ class CogroupedArrowPythonRunner( rightSchema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BaseArrowPythonRunner[(Iterator[InternalRow], Iterator[InternalRow])]( - funcs, evalType, argOffsets) { + extends BasePythonRunner[ + (Iterator[InternalRow], Iterator[InternalRow]), ColumnarBatch](funcs, evalType, argOffsets) + with PythonArrowOutput { protected def newWriterThread( env: SparkEnv, @@ -81,11 +82,11 @@ class CogroupedArrowPythonRunner( dataOut.writeInt(0) } - def writeGroup( + private def writeGroup( group: Iterator[InternalRow], schema: StructType, dataOut: DataOutputStream, - name: String) = { + name: String): Unit = { val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) val allocator = ArrowUtils.rootAllocator.newChildAllocator( s"stdout writer for $pythonExec ($name)", 0, Long.MaxValue) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala index cc83e0cecdc33..b079405bdc2f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.execution.python -import org.apache.spark.api.python.PythonEvalType +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.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} +import org.apache.spark.sql.execution.python.PandasGroupUtils._ import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils /** @@ -52,7 +54,14 @@ case class FlatMapCoGroupsInPandasExec( output: Seq[Attribute], left: SparkPlan, right: SparkPlan) - extends BasePandasGroupExec(func, output) with BinaryExecNode { + extends SparkPlan with BinaryExecNode { + + private val sessionLocalTimeZone = conf.sessionLocalTimeZone + private val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + private val pandasFunction = func.asInstanceOf[PythonUDF].func + private val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + + override def producedAttributes: AttributeSet = AttributeSet(output) override def outputPartitioning: Partitioning = left.outputPartitioning @@ -81,7 +90,7 @@ case class FlatMapCoGroupsInPandasExec( val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) .map { case (_, l, r) => (l, r) } - val runner = new CogroupedArrowPythonRunner( + val runner = new CoGroupedArrowPythonRunner( chainedFunc, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, Array(leftArgOffsets ++ rightArgOffsets), @@ -90,7 +99,7 @@ case class FlatMapCoGroupsInPandasExec( sessionLocalTimeZone, pythonRunnerConf) - executePython(data, runner) + executePython(data, output, runner) } } } 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 22a0d1e09b12e..5032bc81327b9 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 @@ -17,13 +17,15 @@ package org.apache.spark.sql.execution.python -import org.apache.spark.api.python.PythonEvalType +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.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.python.PandasGroupUtils._ import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils /** @@ -48,7 +50,14 @@ case class FlatMapGroupsInPandasExec( func: Expression, output: Seq[Attribute], child: SparkPlan) - extends BasePandasGroupExec(func, output) with UnaryExecNode { + extends SparkPlan with UnaryExecNode { + + private val sessionLocalTimeZone = conf.sessionLocalTimeZone + private val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + private val pandasFunction = func.asInstanceOf[PythonUDF].func + private val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + + override def producedAttributes: AttributeSet = AttributeSet(output) override def outputPartitioning: Partitioning = child.outputPartitioning @@ -72,7 +81,7 @@ case class FlatMapGroupsInPandasExec( inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { val data = groupAndProject(iter, groupingAttributes, child.output, dedupAttributes) - .map{case(_, x) => x} + .map { case (_, x) => x } val runner = new ArrowPythonRunner( chainedFunc, @@ -82,7 +91,7 @@ case class FlatMapGroupsInPandasExec( sessionLocalTimeZone, pythonRunnerConf) - executePython(data, runner) + executePython(data, output, runner) }} } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala similarity index 85% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala index 477c288ad1211..68ce991a8ae7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PandasGroupUtils.scala @@ -21,37 +21,23 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.TaskContext -import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} +import org.apache.spark.api.python.BasePythonRunner import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} -import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} /** * Base functionality for plans which execute grouped python udfs. */ -abstract class BasePandasGroupExec( - func: Expression, - output: Seq[Attribute]) - extends SparkPlan { - - protected val sessionLocalTimeZone = conf.sessionLocalTimeZone - - protected val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) - - protected val pandasFunction = func.asInstanceOf[PythonUDF].func - - protected val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) - - override def producedAttributes: AttributeSet = AttributeSet(output) - +private[python] object PandasGroupUtils { /** * passes the data to the python runner and coverts the resulting * columnarbatch into internal rows. */ - protected def executePython[T]( + def executePython[T]( data: Iterator[T], + output: Seq[Attribute], runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { val context = TaskContext.get() @@ -71,7 +57,7 @@ abstract class BasePandasGroupExec( /** * groups according to grouping attributes and then projects into the deduplicated schema */ - protected def groupAndProject( + def groupAndProject( input: Iterator[InternalRow], groupingAttributes: Seq[Attribute], inputSchema: Seq[Attribute], @@ -101,7 +87,7 @@ abstract class BasePandasGroupExec( * * argOffsets[argOffsets[0]+2 .. ] is the arg offsets for data attributes */ - protected def resolveArgOffsets( + def resolveArgOffsets( child: SparkPlan, groupingAttributes: Seq[Attribute]): (Seq[Attribute], Array[Int]) = { val dataAttributes = child.output.drop(groupingAttributes.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala similarity index 88% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala index 0cee7d2f96c22..bb353062384a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonArrowOutput.scala @@ -16,8 +16,8 @@ */ package org.apache.spark.sql.execution.python -import java.io._ -import java.net._ +import java.io.DataInputStream +import java.net.Socket import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ @@ -25,22 +25,19 @@ import scala.collection.JavaConverters._ import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.ArrowStreamReader -import org.apache.spark._ -import org.apache.spark.api.python._ +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.api.python.{BasePythonRunner, SpecialLengths} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} /** - * Common functionality for a udf runner that exchanges data with Python worker via Arrow stream. + * A trait that can be mixed-in with [[BasePythonRunner]]. It implements the logic from + * Python (Arrow) to JVM (ColumnarBatch). */ -abstract class BaseArrowPythonRunner[T]( - funcs: Seq[ChainedPythonFunctions], - evalType: Int, - argOffsets: Array[Array[Int]]) - extends BasePythonRunner[T, ColumnarBatch](funcs, evalType, argOffsets) { +private[python] trait PythonArrowOutput { self: BasePythonRunner[_, ColumnarBatch] => - protected override def newReaderIterator( + protected def newReaderIterator( stream: DataInputStream, writerThread: WriterThread, startTime: Long, 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 cad89dedb8b07..f54c4b8f22066 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 @@ -304,7 +304,7 @@ case class WindowInPandasExec( var nextRow: UnsafeRow = null var nextGroup: UnsafeRow = null var nextRowAvailable: Boolean = false - private[this] def fetchNextRow() { + private[this] def fetchNextRow(): Unit = { nextRowAvailable = stream.hasNext if (nextRowAvailable) { nextRow = stream.next().asInstanceOf[UnsafeRow] @@ -325,7 +325,7 @@ case class WindowInPandasExec( val frames = factories.map(_(indexRow)) - private[this] def fetchNextPartition() { + private[this] def fetchNextPartition(): Unit = { // 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() 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 67e26dc1a2dbc..92eef6af2238c 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 @@ -262,7 +262,7 @@ class FileStreamSource( // and the value of the maxFileAge parameter. } - override def stop() {} + override def stop(): Unit = {} } 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 dda9d41f630e6..d191a79187f28 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 @@ -21,6 +21,7 @@ import java.sql.Date import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.streaming.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout} import org.apache.spark.unsafe.types.CalendarInterval @@ -159,7 +160,7 @@ private[sql] class GroupStateImpl[S] private( def getTimeoutTimestamp: Long = timeoutTimestamp private def parseDuration(duration: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(duration) + val cal = IntervalUtils.fromString(duration) if (cal.milliseconds < 0 || cal.months < 0) { throw new IllegalArgumentException(s"Provided duration ($duration) is not positive") } 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 af52af0d1d7e6..b8e18b89b54bc 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 @@ -77,7 +77,8 @@ class IncrementalExecution( */ override lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { - sparkSession.sessionState.optimizer.execute(withCachedData) transformAllExpressions { + sparkSession.sessionState.optimizer.executeAndTrack(withCachedData, + tracker) transformAllExpressions { case ts @ CurrentBatchTimestamp(timestamp, _, _) => logInfo(s"Current batch timestamp = $timestamp") ts.toLiteral 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 916bd2ddbc818..f6cc8116c6c4c 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming +import java.io.IOException import java.util.UUID import scala.collection.mutable.ArrayBuffer @@ -43,6 +44,8 @@ class ManifestFileCommitProtocol(jobId: String, path: String) @transient private var fileLog: FileStreamSinkLog = _ private var batchId: Long = _ + @transient private var pendingCommitFiles: ArrayBuffer[Path] = _ + /** * Sets up the manifest log output and the batch id for this job. * Must be called before any other function. @@ -54,13 +57,21 @@ class ManifestFileCommitProtocol(jobId: String, path: String) override def setupJob(jobContext: JobContext): Unit = { require(fileLog != null, "setupManifestOptions must be called before this function") - // Do nothing + pendingCommitFiles = new ArrayBuffer[Path] } override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { require(fileLog != null, "setupManifestOptions must be called before this function") val fileStatuses = taskCommits.flatMap(_.obj.asInstanceOf[Seq[SinkFileStatus]]).toArray + // We shouldn't remove the files if they're written to the metadata: + // `fileLog.add(batchId, fileStatuses)` could fail AFTER writing files to the metadata + // as well as there could be race + // so for the safety we clean up the list before calling anything incurs exception. + // The case is uncommon and we do best effort instead of guarantee, so the simplicity of + // logic here would be OK, and safe for dealing with unexpected situations. + pendingCommitFiles.clear() + if (fileLog.add(batchId, fileStatuses)) { logInfo(s"Committed batch $batchId") } else { @@ -70,7 +81,29 @@ class ManifestFileCommitProtocol(jobId: String, path: String) override def abortJob(jobContext: JobContext): Unit = { require(fileLog != null, "setupManifestOptions must be called before this function") - // Do nothing + // Best effort cleanup of complete files from failed job. + // Since the file has UUID in its filename, we are safe to try deleting them + // as the file will not conflict with file with another attempt on the same task. + if (pendingCommitFiles.nonEmpty) { + pendingCommitFiles.foreach { path => + try { + val fs = path.getFileSystem(jobContext.getConfiguration) + // this is to make sure the file can be seen from driver as well + if (fs.exists(path)) { + fs.delete(path, false) + } + } catch { + case e: IOException => + logWarning(s"Fail to remove temporary file $path, continue removing next.", e) + } + } + pendingCommitFiles.clear() + } + } + + override def onTaskCommit(taskCommit: TaskCommitMessage): Unit = { + pendingCommitFiles ++= taskCommit.obj.asInstanceOf[Seq[SinkFileStatus]] + .map(_.toFileStatus.getPath) } override def setupTask(taskContext: TaskAttemptContext): Unit = { 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 debf1fad57308..f470ad3f9c690 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 @@ -578,8 +578,10 @@ abstract class StreamExecution( protected def getBatchDescriptionString: String = { val batchDescription = if (currentBatchId < 0) "init" else currentBatchId.toString - Option(name).map(_ + "
").getOrElse("") + - s"id = $id
runId = $runId
batch = $batchDescription" + s"""|${Option(name).getOrElse("")} + |id = $id + |runId = $runId + |batch = $batchDescription""".stripMargin } protected def createStreamingWrite( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 7dd491ede9d05..1b8d69ffb7521 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -63,7 +63,7 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) * are dispatched to Spark listener bus. This method is guaranteed to be called by queries in * the same SparkSession as this listener. */ - def post(event: StreamingQueryListener.Event) { + def post(event: StreamingQueryListener.Event): Unit = { event match { case s: QueryStartedEvent => activeQueryRunIds.synchronized { activeQueryRunIds += s.runId } 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 2bdb3402c14b1..daa70a12ba0e7 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 @@ -21,8 +21,8 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.Duration +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval private object Triggers { def validate(intervalMs: Long): Unit = { @@ -30,7 +30,7 @@ private object Triggers { } def convert(interval: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) + val cal = IntervalUtils.fromString(interval) if (cal.months > 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala index 6fcd6ba64f65d..558b4313d6d88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala @@ -86,7 +86,7 @@ class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDat Iterator() } - override def clearDependencies() { + override def clearDependencies(): Unit = { super.clearDependencies() prev = null } 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 26cf1fb8def56..911a526428cf4 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 @@ -237,7 +237,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) lastOffsetCommitted = newOffset } - override def stop() {} + override def stop(): Unit = {} def reset(): Unit = synchronized { batches.clear() 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 22e3f8e035991..4215d62bfb780 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 @@ -193,7 +193,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { ) } val executedPlan = new QueryExecution(sparkSession, query).executedPlan - InSubqueryExec(expr, SubqueryExec(s"subquery${exprId.id}", executedPlan), exprId) + InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala index 241001a857c8f..c6e7f3978469d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala @@ -133,7 +133,7 @@ class SparkPlanGraphNodeWrapper( val cluster: SparkPlanGraphClusterWrapper) { def toSparkPlanGraphNode(): SparkPlanGraphNode = { - assert(node == null ^ cluster == null, "One and only of of nore or cluster must be set.") + assert(node == null ^ cluster == null, "Exactly one of node, cluster values to be set.") if (node != null) node else cluster.toSparkPlanGraphCluster() } 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 89f6edda2ef57..d191f3790ffa8 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 @@ -123,7 +123,7 @@ case class WindowExec( var nextRow: UnsafeRow = null var nextGroup: UnsafeRow = null var nextRowAvailable: Boolean = false - private[this] def fetchNextRow() { + private[this] def fetchNextRow(): Unit = { nextRowAvailable = stream.hasNext if (nextRowAvailable) { nextRow = stream.next().asInstanceOf[UnsafeRow] @@ -144,7 +144,7 @@ case class WindowExec( val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType)) val frames = factories.map(_(windowFunctionResult)) val numFrames = frames.length - private[this] def fetchNextPartition() { + private[this] def fetchNextPartition(): Unit = { // 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() 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 a141b5e6423aa..2ba34647dbca8 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 @@ -3386,6 +3386,177 @@ object functions { ArrayExcept(col1.expr, col2.expr) } + private def createLambda(f: Column => Column) = { + val x = UnresolvedNamedLambdaVariable(Seq("x")) + val function = f(Column(x)).expr + LambdaFunction(function, Seq(x)) + } + + private def createLambda(f: (Column, Column) => Column) = { + val x = UnresolvedNamedLambdaVariable(Seq("x")) + val y = UnresolvedNamedLambdaVariable(Seq("y")) + val function = f(Column(x), Column(y)).expr + LambdaFunction(function, Seq(x, y)) + } + + private def createLambda(f: (Column, Column, Column) => Column) = { + val x = UnresolvedNamedLambdaVariable(Seq("x")) + val y = UnresolvedNamedLambdaVariable(Seq("y")) + val z = UnresolvedNamedLambdaVariable(Seq("z")) + val function = f(Column(x), Column(y), Column(z)).expr + LambdaFunction(function, Seq(x, y, z)) + } + + /** + * Returns an array of elements after applying a tranformation to each element + * in the input array. + * + * @group collection_funcs + * @since 3.0.0 + */ + def transform(column: Column, f: Column => Column): Column = withExpr { + ArrayTransform(column.expr, createLambda(f)) + } + + /** + * Returns an array of elements after applying a tranformation to each element + * in the input array. + * + * @group collection_funcs + * @since 3.0.0 + */ + def transform(column: Column, f: (Column, Column) => Column): Column = withExpr { + ArrayTransform(column.expr, createLambda(f)) + } + + /** + * Returns whether a predicate holds for one or more elements in the array. + * + * @group collection_funcs + * @since 3.0.0 + */ + def exists(column: Column, f: Column => Column): Column = withExpr { + ArrayExists(column.expr, createLambda(f)) + } + + /** + * Returns whether a predicate holds for every element in the array. + * + * @group collection_funcs + * @since 3.0.0 + */ + def forall(column: Column, f: Column => Column): Column = withExpr { + ArrayForAll(column.expr, createLambda(f)) + } + + /** + * Returns an array of elements for which a predicate holds in a given array. + * + * @group collection_funcs + * @since 3.0.0 + */ + def filter(column: Column, f: Column => Column): Column = withExpr { + ArrayFilter(column.expr, createLambda(f)) + } + + /** + * Returns an array of elements for which a predicate holds in a given array. + * + * @group collection_funcs + * @since 3.0.0 + */ + def filter(column: Column, f: (Column, Column) => Column): Column = withExpr { + ArrayFilter(column.expr, createLambda(f)) + } + + /** + * Applies a binary operator to an initial state and all elements in the array, + * and reduces this to a single state. The final state is converted into the final result + * by applying a finish function. + * + * @group collection_funcs + * @since 3.0.0 + */ + def aggregate( + expr: Column, + zero: Column, + merge: (Column, Column) => Column, + finish: Column => Column): Column = withExpr { + ArrayAggregate( + expr.expr, + zero.expr, + createLambda(merge), + createLambda(finish) + ) + } + + /** + * Applies a binary operator to an initial state and all elements in the array, + * and reduces this to a single state. + * + * @group collection_funcs + * @since 3.0.0 + */ + def aggregate(expr: Column, zero: Column, merge: (Column, Column) => Column): Column = + aggregate(expr, zero, merge, c => c) + + /** + * Merge two given arrays, element-wise, into a signle array using a function. + * If one array is shorter, nulls are appended at the end to match the length of the longer + * array, before applying the function. + * + * @group collection_funcs + * @since 3.0.0 + */ + def zip_with(left: Column, right: Column, f: (Column, Column) => Column): Column = withExpr { + ZipWith(left.expr, right.expr, createLambda(f)) + } + + /** + * Applies a function to every key-value pair in a map and returns + * a map with the results of those applications as the new keys for the pairs. + * + * @group collection_funcs + * @since 3.0.0 + */ + def transform_keys(expr: Column, f: (Column, Column) => Column): Column = withExpr { + TransformKeys(expr.expr, createLambda(f)) + } + + /** + * Applies a function to every key-value pair in a map and returns + * a map with the results of those applications as the new values for the pairs. + * + * @group collection_funcs + * @since 3.0.0 + */ + def transform_values(expr: Column, f: (Column, Column) => Column): Column = withExpr { + TransformValues(expr.expr, createLambda(f)) + } + + /** + * Returns a map whose key-value pairs satisfy a predicate. + * + * @group collection_funcs + * @since 3.0.0 + */ + def map_filter(expr: Column, f: (Column, Column) => Column): Column = withExpr { + MapFilter(expr.expr, createLambda(f)) + } + + /** + * Merge two given maps, key-wise into a single map using a function. + * + * @group collection_funcs + * @since 3.0.0 + */ + def map_zip_with( + left: Column, + right: Column, + f: (Column, Column, Column) => Column): Column = withExpr { + MapZipWith(left.expr, right.expr, createLambda(f)) + } + /** * 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 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 20b0143f098c2..91c693ab34c8e 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 @@ -19,12 +19,13 @@ package org.apache.spark.sql.internal import org.apache.spark.SparkConf import org.apache.spark.annotation.Unstable import org.apache.spark.sql.{ExperimentalMethods, SparkSession, UDFRegistration, _} -import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry, ResolveSessionCatalog} 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.connector.catalog.CatalogManager import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.datasources._ @@ -153,6 +154,8 @@ abstract class BaseSessionStateBuilder( protected lazy val v2SessionCatalog = new V2SessionCatalog(catalog, conf) + protected lazy val catalogManager = new CatalogManager(conf, v2SessionCatalog, catalog) + /** * Interface exposed to the user for registering user-defined functions. * @@ -166,12 +169,12 @@ abstract class BaseSessionStateBuilder( * * Note: this depends on the `conf` and `catalog` fields. */ - protected def analyzer: Analyzer = new Analyzer(catalog, v2SessionCatalog, conf) { + protected def analyzer: Analyzer = new Analyzer(catalogManager, conf) { override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - DataSourceResolution(conf, this.catalogManager) +: + new ResolveSessionCatalog(catalogManager, conf, catalog.isView) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = @@ -225,7 +228,7 @@ abstract class BaseSessionStateBuilder( * Note: this depends on `catalog` and `experimentalMethods` fields. */ protected def optimizer: Optimizer = { - new SparkOptimizer(catalog, experimentalMethods) { + new SparkOptimizer(catalogManager, catalog, experimentalMethods) { override def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = super.extendedOperatorOptimizationRules ++ customOperatorOptimizationRules } @@ -313,7 +316,7 @@ abstract class BaseSessionStateBuilder( () => analyzer, () => optimizer, planner, - streamingQueryManager, + () => streamingQueryManager, listenerManager, () => resourceLoader, createQueryExecution, 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 92e255ae8aaa8..abd1250628539 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 @@ -49,7 +49,8 @@ import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListen * unresolved attributes and relations. * @param optimizerBuilder a function to create the logical query plan optimizer. * @param planner Planner that converts optimized logical plans to physical plans. - * @param streamingQueryManager Interface to start and stop streaming queries. + * @param streamingQueryManagerBuilder A function to create a streaming query manager to + * start and stop streaming queries. * @param listenerManager Interface to register custom [[QueryExecutionListener]]s. * @param resourceLoaderBuilder a function to create a session shared resource loader to load JARs, * files, etc. @@ -67,7 +68,7 @@ private[sql] class SessionState( analyzerBuilder: () => Analyzer, optimizerBuilder: () => Optimizer, val planner: SparkPlanner, - val streamingQueryManager: StreamingQueryManager, + val streamingQueryManagerBuilder: () => StreamingQueryManager, val listenerManager: ExecutionListenerManager, resourceLoaderBuilder: () => SessionResourceLoader, createQueryExecution: LogicalPlan => QueryExecution, @@ -83,6 +84,10 @@ private[sql] class SessionState( lazy val resourceLoader: SessionResourceLoader = resourceLoaderBuilder() + // The streamingQueryManager is lazy to avoid creating a StreamingQueryManager for each session + // when connecting to ThriftServer. + lazy val streamingQueryManager: StreamingQueryManager = streamingQueryManagerBuilder() + def catalogManager: CatalogManager = analyzer.catalogManager def newHadoopConf(): Configuration = SessionState.newHadoopConf( 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 f1a648176c3b3..d097f9f18f89b 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 @@ -18,7 +18,8 @@ package org.apache.spark.sql.internal import java.net.URL -import java.util.Locale +import java.util.{Locale, UUID} +import java.util.concurrent.ConcurrentHashMap import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -33,6 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ 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.sql.streaming.StreamingQueryManager import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils @@ -110,6 +112,12 @@ private[sql] class SharedState( */ val cacheManager: CacheManager = new CacheManager + /** + * A map of active streaming queries to the session specific StreamingQueryManager that manages + * the lifecycle of that stream. + */ + private[sql] val activeStreamingQueries = new ConcurrentHashMap[UUID, StreamingQueryManager]() + /** * A status store to query SQL status/metrics of this Spark application, based on SQL-specific * [[org.apache.spark.scheduler.SparkListenerEvent]]s. 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 9abe38dfda0be..9b43a83e7b94a 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 @@ -352,8 +352,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo } } - // Make sure no other query with same id is active - if (activeQueries.values.exists(_.id == query.id)) { + // Make sure no other query with same id is active across all sessions + val activeOption = + Option(sparkSession.sharedState.activeStreamingQueries.putIfAbsent(query.id, this)) + if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id)) { throw new IllegalStateException( s"Cannot start query with id ${query.id} as another query with same id is " + s"already active. Perhaps you are attempting to restart a query from checkpoint " + @@ -370,9 +372,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo query.streamingQuery.start() } catch { case e: Throwable => - activeQueriesLock.synchronized { - activeQueries -= query.id - } + unregisterTerminatedStream(query.id) throw e } query @@ -380,9 +380,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo /** Notify (by the StreamingQuery) that the query has been terminated */ private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { - activeQueriesLock.synchronized { - activeQueries -= terminatedQuery.id - } + unregisterTerminatedStream(terminatedQuery.id) awaitTerminationLock.synchronized { if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { lastTerminatedQuery = terminatedQuery @@ -391,4 +389,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo } stateStoreCoordinator.deactivateInstances(terminatedQuery.runId) } + + private def unregisterTerminatedStream(terminatedQueryId: UUID): Unit = { + activeQueriesLock.synchronized { + // remove from shared state only if the streaming query manager also matches + sparkSession.sharedState.activeStreamingQueries.remove(terminatedQueryId, this) + activeQueries -= terminatedQueryId + } + } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java index 72f2e8f185637..e418958bef94d 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java @@ -57,21 +57,18 @@ public void dropTestTable() { public void testAppendAPI() throws NoSuchTableException { df().writeTo("testcat.t").append(); df().writeTo("testcat.t").option("property", "value").append(); - df().writeTo("testcat.t").tableProperty("property", "value").append(); } @Test public void testOverwritePartitionsAPI() throws NoSuchTableException { df().writeTo("testcat.t").overwritePartitions(); df().writeTo("testcat.t").option("property", "value").overwritePartitions(); - df().writeTo("testcat.t").tableProperty("property", "value").overwritePartitions(); } @Test public void testOverwriteAPI() throws NoSuchTableException { df().writeTo("testcat.t").overwrite(lit(true)); df().writeTo("testcat.t").option("property", "value").overwrite(lit(true)); - df().writeTo("testcat.t").tableProperty("property", "value").overwrite(lit(true)); } @Test diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java new file mode 100644 index 0000000000000..e240326bee63a --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.util.HashMap; +import java.util.List; + +import static scala.collection.JavaConverters.mapAsScalaMap; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.*; +import static org.apache.spark.sql.types.DataTypes.*; +import static org.apache.spark.sql.functions.*; +import org.apache.spark.sql.test.TestSparkSession; +import static test.org.apache.spark.sql.JavaTestUtils.*; + +public class JavaHigherOrderFunctionsSuite { + private transient TestSparkSession spark; + private Dataset arrDf; + private Dataset mapDf; + + private void setUpArrDf() { + List data = toRows( + makeArray(1, 9, 8, 7), + makeArray(5, 8, 9, 7, 2), + JavaTestUtils.makeArray(), + null + ); + StructType schema = new StructType() + .add("x", new ArrayType(IntegerType, true), true); + arrDf = spark.createDataFrame(data, schema); + } + + private void setUpMapDf() { + List data = toRows( + new HashMap() {{ + put(1, 1); + put(2, 2); + }}, + null + ); + StructType schema = new StructType() + .add("x", new MapType(IntegerType, IntegerType, true)); + mapDf = spark.createDataFrame(data, schema); + } + + @Before + public void setUp() { + spark = new TestSparkSession(); + setUpArrDf(); + setUpMapDf(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @Test + public void testTransform() { + checkAnswer( + arrDf.select(transform(col("x"), x -> x.plus(1))), + toRows( + makeArray(2, 10, 9, 8), + makeArray(6, 9, 10, 8, 3), + JavaTestUtils.makeArray(), + null + ) + ); + checkAnswer( + arrDf.select(transform(col("x"), (x, i) -> x.plus(i))), + toRows( + makeArray(1, 10, 10, 10), + makeArray(5, 9, 11, 10, 6), + JavaTestUtils.makeArray(), + null + ) + ); + } + + @Test + public void testFilter() { + checkAnswer( + arrDf.select(filter(col("x"), x -> x.plus(1).equalTo(10))), + toRows( + makeArray(9), + makeArray(9), + JavaTestUtils.makeArray(), + null + ) + ); + checkAnswer( + arrDf.select(filter(col("x"), (x, i) -> x.plus(i).equalTo(10))), + toRows( + makeArray(9, 8, 7), + makeArray(7), + JavaTestUtils.makeArray(), + null + ) + ); + } + + @Test + public void testExists() { + checkAnswer( + arrDf.select(exists(col("x"), x -> x.plus(1).equalTo(10))), + toRows( + true, + true, + false, + null + ) + ); + } + + @Test + public void testForall() { + checkAnswer( + arrDf.select(forall(col("x"), x -> x.plus(1).equalTo(10))), + toRows( + false, + false, + true, + null + ) + ); + } + + @Test + public void testAggregate() { + checkAnswer( + arrDf.select(aggregate(col("x"), lit(0), (acc, x) -> acc.plus(x))), + toRows( + 25, + 31, + 0, + null + ) + ); + checkAnswer( + arrDf.select(aggregate(col("x"), lit(0), (acc, x) -> acc.plus(x), x -> x)), + toRows( + 25, + 31, + 0, + null + ) + ); + } + + @Test + public void testZipWith() { + checkAnswer( + arrDf.select(zip_with(col("x"), col("x"), (a, b) -> lit(42))), + toRows( + makeArray(42, 42, 42, 42), + makeArray(42, 42, 42, 42, 42), + JavaTestUtils.makeArray(), + null + ) + ); + } + + @Test + public void testTransformKeys() { + checkAnswer( + mapDf.select(transform_keys(col("x"), (k, v) -> k.plus(v))), + toRows( + mapAsScalaMap(new HashMap() {{ + put(2, 1); + put(4, 2); + }}), + null + ) + ); + } + + @Test + public void testTransformValues() { + checkAnswer( + mapDf.select(transform_values(col("x"), (k, v) -> k.plus(v))), + toRows( + mapAsScalaMap(new HashMap() {{ + put(1, 2); + put(2, 4); + }}), + null + ) + ); + } + + @Test + public void testMapFilter() { + checkAnswer( + mapDf.select(map_filter(col("x"), (k, v) -> lit(false))), + toRows( + mapAsScalaMap(new HashMap()), + null + ) + ); + } + + @Test + public void testMapZipWith() { + checkAnswer( + mapDf.select(map_zip_with(col("x"), col("x"), (k, v1, v2) -> lit(false))), + toRows( + mapAsScalaMap(new HashMap() {{ + put(1, false); + put(2, false); + }}), + null + ) + ); + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala b/sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java similarity index 50% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java index bba7f12c94e50..7fc6460e7352c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java @@ -15,19 +15,33 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.plans.logical.sql +package test.org.apache.spark.sql; -/** - * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterViewSetPropertiesStatement( - viewName: Seq[String], - properties: Map[String, String]) extends ParsedStatement +import java.util.Arrays; +import java.util.List; +import static java.util.stream.Collectors.toList; -/** - * ALTER VIEW ... UNSET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterViewUnsetPropertiesStatement( - viewName: Seq[String], - propertyKeys: Seq[String], - ifExists: Boolean) extends ParsedStatement +import scala.collection.mutable.WrappedArray; + +import static org.junit.Assert.assertEquals; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; + +public class JavaTestUtils { + public static void checkAnswer(Dataset actual, List expected) { + assertEquals(expected, actual.collectAsList()); + } + + public static List toRows(Object... objs) { + return Arrays.asList(objs) + .stream() + .map(RowFactory::create) + .collect(toList()); + } + + public static WrappedArray makeArray(T... ts) { + return WrappedArray.make(ts); + } +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql new file mode 100644 index 0000000000000..993eecf0f89b6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql @@ -0,0 +1,39 @@ +-- test cases for bitwise functions + +-- null +select bit_count(null); + +-- boolean +select bit_count(true); +select bit_count(false); + +-- byte/tinyint +select bit_count(cast(1 as tinyint)); +select bit_count(cast(2 as tinyint)); +select bit_count(cast(3 as tinyint)); + +-- short/smallint +select bit_count(1S); +select bit_count(2S); +select bit_count(3S); + +-- int +select bit_count(1); +select bit_count(2); +select bit_count(3); + +-- long/bigint +select bit_count(1L); +select bit_count(2L); +select bit_count(3L); + +-- negative num +select bit_count(-1L); + +-- edge value +select bit_count(9223372036854775807L); +select bit_count(-9223372036854775808L); + +-- other illegal arguments +select bit_count("bit count"); +select bit_count('a'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql index fd0fb50f71460..a63cdafb745a0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql @@ -68,3 +68,78 @@ select date_part('not_supported', c) from t; select date_part(c, c) from t; select date_part(null, c) from t; + +CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c; + +select date_part('millennium', c) from t2; +select date_part('millennia', c) from t2; +select date_part('mil', c) from t2; +select date_part('mils', c) from t2; + +select date_part('century', c) from t2; +select date_part('centuries', c) from t2; +select date_part('c', c) from t2; +select date_part('cent', c) from t2; + +select date_part('decade', c) from t2; +select date_part('decades', c) from t2; +select date_part('dec', c) from t2; +select date_part('decs', c) from t2; + +select date_part('year', c) from t2; +select date_part('y', c) from t2; +select date_part('years', c) from t2; +select date_part('yr', c) from t2; +select date_part('yrs', c) from t2; + +select date_part('quarter', c) from t2; +select date_part('qtr', c) from t2; + +select date_part('month', c) from t2; +select date_part('mon', c) from t2; +select date_part('mons', c) from t2; +select date_part('months', c) from t2; + +select date_part('day', c) from t2; +select date_part('d', c) from t2; +select date_part('days', c) from t2; + +select date_part('hour', c) from t2; +select date_part('h', c) from t2; +select date_part('hours', c) from t2; +select date_part('hr', c) from t2; +select date_part('hrs', c) from t2; + +select date_part('minute', c) from t2; +select date_part('m', c) from t2; +select date_part('min', c) from t2; +select date_part('mins', c) from t2; +select date_part('minutes', c) from t2; + +select date_part('second', c) from t2; +select date_part('s', c) from t2; +select date_part('sec', c) from t2; +select date_part('seconds', c) from t2; +select date_part('secs', c) from t2; + +select date_part('milliseconds', c) from t2; +select date_part('msec', c) from t2; +select date_part('msecs', c) from t2; +select date_part('millisecon', c) from t2; +select date_part('mseconds', c) from t2; +select date_part('ms', c) from t2; + +select date_part('microseconds', c) from t2; +select date_part('usec', c) from t2; +select date_part('usecs', c) from t2; +select date_part('useconds', c) from t2; +select date_part('microsecon', c) from t2; +select date_part('us', c) from t2; + +select date_part('epoch', c) from t2; + +select date_part('not_supported', c) from t2; + +select date_part(c, c) from t2; + +select date_part(null, c) from 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 2f7ffb73e86b8..0e22af1fbdf29 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -34,3 +34,5 @@ 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'; +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; 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 66bc90914e0d4..fcde225676cb9 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 @@ -90,16 +90,16 @@ CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (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; +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; -- all null values -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4; +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; -- aggregates are null Filtering -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5; +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; -- group by -SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k; +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; -- having SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; @@ -137,10 +137,18 @@ SELECT any(1L); -- input type checking String SELECT every("true"); --- every/some/any aggregates are supported as windows expression. +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or 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; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(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; diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 816386c483209..0f95f85237828 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -85,6 +85,19 @@ select timestamp '2016-33-11 20:54:00.000'; -- interval select interval 13.123456789 seconds, interval -13.123456789 second; select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond; +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; +select interval '0 0:0:0.1' day to second; +select interval '10-9' year to month; +select interval '20 15:40:32.99899999' day to hour; +select interval '20 15:40:32.99899999' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40:32.99899999' hour to minute; +select interval '15:40.99899999' hour to second; +select interval '15:40' hour to second; +select interval '15:40:32.99899999' hour to second; +select interval '20 40:32.99899999' minute to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; -- ns is not supported select interval 10 nanoseconds; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql similarity index 74% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql index 47f9d2f373069..ba91366014e16 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql @@ -41,42 +41,37 @@ create temporary view int4_tbl as select * from values -- -- test for bitwise integer aggregates -- --- CREATE TEMPORARY TABLE bitwise_test( --- i2 INT2, --- i4 INT4, --- i8 INT8, --- i INTEGER, --- x INT2, --- y BIT(4) --- ); +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); -- 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; +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null; + +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test; + +-- group by +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1; + +--having +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7; + +-- window +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; -- -- test boolean aggregates @@ -114,50 +109,40 @@ SELECT 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); +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4); -- 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; +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0; + +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test; + +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test; + +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test; -- -- Test cases that should be optimized into indexscans instead of diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part3.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part4.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/case.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/case.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/comments.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/comments.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/comments.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/comments.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql new file mode 100644 index 0000000000000..eb8cc34419519 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql @@ -0,0 +1,344 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INTERVAL +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/interval.sql + +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET DATESTYLE = 'ISO'; +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres; + +-- check acceptance of "time zone style" +-- [SPARK-29369] Accept strings without `interval` prefix in casting to intervals +-- [SPARK-29370] Interval strings without explicit unit markings +-- SELECT INTERVAL '01:00' AS `One hour`; +-- SELECT INTERVAL '+02:00' AS `Two hours`; +-- SELECT INTERVAL '-08:00' AS `Eight hours`; +-- SELECT INTERVAL '-1 +02:03' AS `22 hours ago...`; +-- SELECT INTERVAL '-1 days +02:03' AS `22 hours ago...`; +-- [SPARK-29371] Support interval field values with fractional parts +-- SELECT INTERVAL '1.5 weeks' AS `Ten days twelve hours`; +-- SELECT INTERVAL '1.5 months' AS `One month 15 days`; +-- SELECT INTERVAL '10 years -11 month -12 days +13:14' AS `9 years...`; + +-- [SPARK-29382] Support writing `INTERVAL` type to datasource table +-- CREATE TABLE INTERVAL_TBL (f1 interval); + +-- [SPARK-29383] Support the optional prefix `@` in interval strings +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 1 minute'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 5 hour'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 10 day'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 34 year'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 3 months'); +-- [SPARK-29384] Support `ago` in interval strings +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 14 seconds ago'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('1 day 2 hours 3 minutes 4 seconds'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('6 years'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('5 months'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('5 months 12 hours'); + +-- badly formatted interval +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('badly formatted interval'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 30 eons ago'); + +-- test interval operators + +-- SELECT '' AS ten, * FROM INTERVAL_TBL; +-- [SPARK-29385] Make `INTERVAL` values comparable +-- SELECT '' AS nine, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 <> interval '@ 10 days'; + +-- SELECT '' AS three, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 <= interval '@ 5 hours'; + +-- SELECT '' AS three, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 < interval '@ 1 day'; + +-- SELECT '' AS one, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 = interval '@ 34 years'; + +-- SELECT '' AS five, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 >= interval '@ 1 month'; + +-- SELECT '' AS nine, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 > interval '@ 3 seconds ago'; + +-- SELECT '' AS fortyfive, r1.*, r2.* +-- FROM INTERVAL_TBL r1, INTERVAL_TBL r2 +-- WHERE r1.f1 > r2.f1 +-- ORDER BY r1.f1, r2.f1; + +-- Test intervals that are large enough to overflow 64 bits in comparisons +-- [SPARK-29369] Accept strings without `interval` prefix in casting to intervals +-- CREATE TEMP TABLE INTERVAL_TBL_OF (f1 interval); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES +-- ('2147483647 days 2147483647 months'), +-- ('2147483647 days -2147483648 months'), +-- ('1 year'), +-- ('-2147483648 days 2147483647 months'), +-- ('-2147483648 days -2147483648 months'); +-- these should fail as out-of-range +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483648 days'); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483649 days'); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483647 years'); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483648 years'); + +-- SELECT r1.*, r2.* +-- FROM INTERVAL_TBL_OF r1, INTERVAL_TBL_OF r2 +-- WHERE r1.f1 > r2.f1 +-- ORDER BY r1.f1, r2.f1; + +-- CREATE INDEX ON INTERVAL_TBL_OF USING btree (f1); +-- SET enable_seqscan TO false; +-- EXPLAIN (COSTS OFF) +-- SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; +-- SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; +-- RESET enable_seqscan; + +-- DROP TABLE INTERVAL_TBL_OF; + +-- Test multiplication and division with intervals. +-- Floating point arithmetic rounding errors can lead to unexpected results, +-- though the code attempts to do the right thing and round up to days and +-- minutes to avoid results such as '3 days 24:00 hours' or '14:20:60'. +-- Note that it is expected for some day components to be greater than 29 and +-- some time components be greater than 23:59:59 due to how intervals are +-- stored internally. +-- [SPARK-29386] Copy data between a file and a table +-- CREATE TABLE INTERVAL_MULDIV_TBL (span interval); +-- COPY INTERVAL_MULDIV_TBL FROM STDIN; +-- 41 mon 12 days 360:00 +-- -41 mon -12 days +360:00 +-- -12 days +-- 9 mon -27 days 12:34:56 +-- -3 years 482 days 76:54:32.189 +-- 4 mon +-- 14 mon +-- 999 mon 999 days +-- \. +-- [SPARK-29387] Support `*` and `\` operators for intervals +-- SELECT span * 0.3 AS product +-- FROM INTERVAL_MULDIV_TBL; + +-- SELECT span * 8.2 AS product +-- FROM INTERVAL_MULDIV_TBL; + +-- SELECT span / 10 AS quotient +-- FROM INTERVAL_MULDIV_TBL; + +-- SELECT span / 100 AS quotient +-- FROM INTERVAL_MULDIV_TBL; + +-- DROP TABLE INTERVAL_MULDIV_TBL; +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET DATESTYLE = 'postgres'; +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres_verbose; + +-- SELECT '' AS ten, * FROM INTERVAL_TBL; + +-- test avg(interval), which is somewhat fragile since people have been +-- known to change the allowed input syntax for type interval without +-- updating pg_aggregate.agginitval + +-- select avg(f1) from interval_tbl; + +-- test long interval input +-- [SPARK-29388] Construct intervals from the `millenniums`, `centuries` or `decades` units +-- select '4 millenniums 5 centuries 4 decades 1 year 4 months 4 days 17 minutes 31 seconds'::interval; + +-- test long interval output +-- Note: the actual maximum length of the interval output is longer, +-- but we need the test to work for both integer and floating-point +-- timestamps. +-- [SPARK-29389] Support synonyms for interval units +-- select '100000000y 10mon -1000000000d -100000h -10min -10.000001s ago'::interval; + +-- test justify_hours() and justify_days() +-- [SPARK-29390] Add the justify_days(), justify_hours() and justify_interval() functions +-- SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as `6 mons 5 days 4 hours 3 mins 2 seconds`; +-- SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as `7 mons 6 days 5 hours 4 mins 3 seconds`; + +-- test justify_interval() + +-- SELECT justify_interval(interval '1 month -1 hour') as `1 month -1 hour`; + +-- test fractional second input, and detection of duplicate units +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET DATESTYLE = 'ISO'; +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO postgres; +-- [SPARK-29369] Accept strings without `interval` prefix in casting to intervals +-- SELECT '1 millisecond'::interval, '1 microsecond'::interval, +-- '500 seconds 99 milliseconds 51 microseconds'::interval; +-- SELECT '3 days 5 milliseconds'::interval; + +-- SELECT '1 second 2 seconds'::interval; -- error +-- SELECT '10 milliseconds 20 milliseconds'::interval; -- error +-- SELECT '5.5 seconds 3 milliseconds'::interval; -- error +-- SELECT '1:20:05 5 microseconds'::interval; -- error +-- SELECT '1 day 1 day'::interval; -- error +-- [SPARK-29391] Default year-month units +-- SELECT interval '1-2'; -- SQL year-month literal +SELECT interval '999' second; -- oversize leading field is ok +SELECT interval '999' minute; +SELECT interval '999' hour; +SELECT interval '999' day; +SELECT interval '999' month; + +-- test SQL-spec syntaxes for restricted field sets +SELECT interval '1' year; +SELECT interval '2' month; +SELECT interval '3' day; +SELECT interval '4' hour; +SELECT interval '5' minute; +SELECT interval '6' second; +-- [SPARK-29391] Default year-month units +-- SELECT interval '1' year to month; +SELECT interval '1-2' year to month; +-- [SPARK-29391] Default year-month units +-- SELECT interval '1 2' day to hour; +SELECT interval '1 2:03' day to hour; +SELECT interval '1 2:03:04' day to hour; +-- SELECT interval '1 2' day to minute; +SELECT interval '1 2:03' day to minute; +SELECT interval '1 2:03:04' day to minute; +-- SELECT interval '1 2' day to second; +SELECT interval '1 2:03' day to second; +SELECT interval '1 2:03:04' day to second; +-- SELECT interval '1 2' hour to minute; +SELECT interval '1 2:03' hour to minute; +SELECT interval '1 2:03:04' hour to minute; +-- SELECT interval '1 2' hour to second; +SELECT interval '1 2:03' hour to second; +SELECT interval '1 2:03:04' hour to second; +-- SELECT interval '1 2' minute to second; +SELECT interval '1 2:03' minute to second; +SELECT interval '1 2:03:04' minute to second; +-- [SPARK-29370] Interval strings without explicit unit markings +-- SELECT interval '1 +2:03' minute to second; +-- SELECT interval '1 +2:03:04' minute to second; +-- SELECT interval '1 -2:03' minute to second; +-- SELECT interval '1 -2:03:04' minute to second; +-- SELECT interval '123 11' day to hour; -- ok +-- SELECT interval '123 11' day; -- not ok +-- SELECT interval '123 11'; -- not ok, too ambiguous +-- SELECT interval '123 2:03 -2:04'; -- not ok, redundant hh:mm fields + +-- test syntaxes for restricted precision +-- [SPARK-29395] Precision of the interval type +-- SELECT interval(0) '1 day 01:23:45.6789'; +-- SELECT interval(2) '1 day 01:23:45.6789'; +-- SELECT interval '12:34.5678' minute to second(2); -- per SQL spec +-- SELECT interval '1.234' second; +-- SELECT interval '1.234' second(2); +-- SELECT interval '1 2.345' day to second(2); +-- SELECT interval '1 2:03' day to second(2); +-- SELECT interval '1 2:03.4567' day to second(2); +-- SELECT interval '1 2:03:04.5678' day to second(2); +-- SELECT interval '1 2.345' hour to second(2); +-- SELECT interval '1 2:03.45678' hour to second(2); +-- SELECT interval '1 2:03:04.5678' hour to second(2); +-- SELECT interval '1 2.3456' minute to second(2); +-- SELECT interval '1 2:03.5678' minute to second(2); +-- SELECT interval '1 2:03:04.5678' minute to second(2); + +-- test casting to restricted precision (bug #14479) +-- SELECT f1, f1::INTERVAL DAY TO MINUTE AS `minutes`, +-- (f1 + INTERVAL '1 month')::INTERVAL MONTH::INTERVAL YEAR AS `years` +-- FROM interval_tbl; + +-- test inputting and outputting SQL standard interval literals +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO sql_standard; +-- [SPARK-29407] Support syntax for zero interval +-- SELECT interval '0' AS zero, +-- interval '1-2' year to month AS `year-month`, +-- interval '1 2:03:04' day to second AS `day-time`, +-- [SPARK-29408] Support interval literal with negative sign `-` +-- - interval '1-2' AS `negative year-month`, +-- - interval '1 2:03:04' AS `negative day-time`; + +-- test input of some not-quite-standard interval values in the sql style +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO postgres; +-- SELECT interval '+1 -1:00:00', +-- interval '-1 +1:00:00', +-- interval '+1-2 -3 +4:05:06.789', +-- interval '-1-2 +3 -4:05:06.789'; + +-- test output of couple non-standard interval values in the sql style +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO sql_standard; +-- SELECT interval '1 day -1 hours', +-- interval '-1 days +1 hours', +-- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', +-- - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; + +-- test outputting iso8601 intervals +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to iso_8601; +-- select interval '0' AS zero, +-- interval '1-2' AS `a year 2 months`, +-- interval '1 2:03:04' AS `a bit over a day`, +-- interval '2:03:04.45679' AS `a bit over 2 hours`, +-- (interval '1-2' + interval '3 4:05:06.7') AS `all fields`, +-- (interval '1-2' - interval '3 4:05:06.7') AS `mixed sign`, +-- (- interval '1-2' + interval '3 4:05:06.7') AS negative; + +-- test inputting ISO 8601 4.4.2.1 "Format With Time Unit Designators" +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to sql_standard; +-- [SPARK-29394] Support ISO 8601 format for intervals +-- select interval 'P0Y' AS zero, +-- interval 'P1Y2M' AS `a year 2 months`, +-- interval 'P1W' AS `a week`, +-- interval 'P1DT2H3M4S' AS `a bit over a day`, +-- interval 'P1Y2M3DT4H5M6.7S' AS `all fields`, +-- interval 'P-1Y-2M-3DT-4H-5M-6.7S' AS negative, +-- interval 'PT-0.1S' AS `fractional second`; + +-- test inputting ISO 8601 4.4.2.2 "Alternative Format" +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres; +-- select interval 'P00021015T103020' AS `ISO8601 Basic Format`, +-- interval 'P0002-10-15T10:30:20' AS `ISO8601 Extended Format`; + +-- Make sure optional ISO8601 alternative format fields are optional. +-- select interval 'P0002' AS `year only`, +-- interval 'P0002-10' AS `year month`, +-- interval 'P0002-10-15' AS `year month day`, +-- interval 'P0002T1S' AS `year only plus time`, +-- interval 'P0002-10T1S' AS `year month plus time`, +-- interval 'P0002-10-15T1S' AS `year month day plus time`, +-- interval 'PT10' AS `hour only`, +-- interval 'PT10:30' AS `hour minute`; + +-- test a couple rounding cases that changed since 8.3 w/ HAVE_INT64_TIMESTAMP. +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres_verbose; +-- select interval '-10 mons -3 days +03:55:06.70'; +-- select interval '1 year 2 mons 3 days 04:05:06.699999'; +-- select interval '0:0:0.7', interval '@ 0.70 secs', interval '0.7 seconds'; + +-- check that '30 days' equals '1 month' according to the hash function +-- [SPARK-29385] Make `INTERVAL` values comparable +-- select '30 days'::interval = '1 month'::interval as t; +-- select interval_hash('30 days'::interval) = interval_hash('1 month'::interval) as t; + +-- numeric constructor +-- [SPARK-29393] Add the make_interval() function +-- select make_interval(years := 2); +-- select make_interval(years := 1, months := 6); +-- select make_interval(years := 1, months := -1, weeks := 5, days := -7, hours := 25, mins := -180); + +-- select make_interval() = make_interval(years := 0, months := 0, weeks := 0, days := 0, mins := 0, secs := 0.0); +-- select make_interval(hours := -2, mins := -10, secs := -25.3); + +-- select make_interval(years := 'inf'::float::int); +-- select make_interval(months := 'NaN'::float::int); +-- select make_interval(secs := 'inf'); +-- select make_interval(secs := 'NaN'); +-- select make_interval(secs := 7e12); diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/join.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql similarity index 99% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/join.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql index 08f54fe0a40e5..1cfc463acc588 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql @@ -577,15 +577,15 @@ select count(*) from tenk1 a, tenk1 b -- regression test for 8.2 bug with improper re-ordering of left joins -- -DROP TABLE IF EXISTS tt3; -CREATE TABLE tt3(f1 int, f2 string) USING parquet; -INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x; +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2); -- create index tt3i on tt3(f1); -- analyze tt3; -DROP TABLE IF EXISTS tt4; -CREATE TABLE tt4(f1 int) USING parquet; -INSERT INTO tt4 VALUES (0),(1),(9999); +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1); -- analyze tt4; SELECT a.f1 diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_distinct.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_distinct.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_implicit.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_implicit.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/strings.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/strings.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/strings.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/strings.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql similarity index 96% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql index 92bbe14dc1e5f..260e8ea93d22d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql @@ -175,18 +175,16 @@ SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL 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 `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'; +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'; SELECT '' AS `54`, d1 as `timestamp`, date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/union.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/union.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/union.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/union.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql new file mode 100644 index 0000000000000..ae2a015ada245 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql @@ -0,0 +1,352 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- CREATE TABLE empsalary ( +-- depname string, +-- empno integer, +-- salary int, +-- enroll_date date +-- ) USING parquet; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- INSERT INTO empsalary VALUES ('develop', 10, 5200, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('sales', 1, 5000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('personnel', 5, 3500, '2007-12-10'); +-- INSERT INTO empsalary VALUES ('sales', 4, 4800, '2007-08-08'); +-- INSERT INTO empsalary VALUES ('personnel', 2, 3900, '2006-12-23'); +-- INSERT INTO empsalary VALUES ('develop', 7, 4200, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('develop', 9, 4500, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('sales', 3, 4800, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('develop', 8, 6000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('develop', 11, 5200, '2007-08-15'); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname) FROM empsalary ORDER BY depname, salary; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, rank() OVER (PARTITION BY depname ORDER BY salary) FROM empsalary; + +-- with GROUP BY +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER w FROM empsalary WINDOW w AS (PARTITION BY depname); + +-- [SPARK-28064] Order by does not accept a call to rank() +-- SELECT depname, empno, salary, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary) ORDER BY rank() OVER w; + +-- empty window specification +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10; + +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS (); + +-- no window operation +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten); + +-- cumulative aggregate +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10; + +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28065] ntile does not accept NULL as input +-- SELECT ntile(NULL) OVER (ORDER BY ten, four), ten, four FROM tenk1 LIMIT 2; + +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four, 0) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- last returns the last row of the frame, which is CURRENT ROW in ORDER BY window. +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- SELECT nth_value(ten, four + 1) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two; + +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10; + +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10; + +-- opexpr with different windows evaluation. +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum; + +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- more than one window with GROUP BY +-- SELECT sum(salary), +-- row_number() OVER (ORDER BY depname), +-- sum(sum(salary)) OVER (ORDER BY depname DESC) +-- FROM empsalary GROUP BY depname; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- identical windows with different names +-- SELECT sum(salary) OVER w1, count(*) OVER w2 +-- FROM empsalary WINDOW w1 AS (ORDER BY salary), w2 AS (ORDER BY salary); + +-- subplan +-- [SPARK-28379] Correlated scalar subqueries must be aggregated +-- SELECT lead(ten, (SELECT two FROM tenk1 WHERE s.unique2 = unique2)) OVER (PARTITION BY four ORDER BY ten) +-- FROM tenk1 s WHERE unique2 < 10; + +-- empty table +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- mixture of agg/wfunc in the same window +-- SELECT sum(salary) OVER w, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary DESC); + +-- Cannot safely cast 'enroll_date': StringType to DateType; +-- SELECT empno, depname, salary, bonus, depadj, MIN(bonus) OVER (ORDER BY empno), MAX(depadj) OVER () FROM( +-- SELECT *, +-- CASE WHEN enroll_date < '2008-01-01' THEN 2008 - extract(year FROM enroll_date) END * 500 AS bonus, +-- CASE WHEN +-- AVG(salary) OVER (PARTITION BY depname) < salary +-- THEN 200 END AS depadj FROM empsalary +-- )s; + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- window function over ungrouped agg over empty row set (bug before 9.1) +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42; + +-- window function with ORDER BY an expression involving aggregates (9.1 bug) +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten; + +-- window and aggregate with GROUP BY expression (9.2 bug) +-- explain +-- select first(max(x)) over (), y +-- from (select unique1 as x, ten+four as y from tenk1) ss +-- group by y; + +-- test non-default frame specifications +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- SELECT first_value(unique1) over w, +-- nth_value(unique1, 2) over w AS nth_2, +-- last_value(unique1) over w, unique1, four +-- FROM tenk1 WHERE unique1 < 10 +-- WINDOW w AS (order by four range between current row and unbounded following); + +-- [SPARK-28501] Frame bound value must be a literal. +-- SELECT sum(unique1) over +-- (order by unique1 +-- rows (SELECT unique1 FROM tenk1 ORDER BY unique1 LIMIT 1) + 1 PRECEDING), +-- unique1 +-- FROM tenk1 WHERE unique1 < 10; + +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i; + +SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude current row) as sum_rows FROM range(1, 10) i; + +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude group) as sum_rows FROM range(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude ties) as sum_rows FROM generate_series(1, 10) i; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude no others) as sum_rows FROM generate_series(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i.id, sum(i.id) over (order by i.id groups between 1 preceding and 1 following) as sum_rows FROM range(1, 11) i; +-- SELECT * FROM v_window; + +DROP VIEW v_window; +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- DROP TABLE empsalary; +DROP VIEW tenk2; +DROP VIEW int4_tbl; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql new file mode 100644 index 0000000000000..728e8cab0c3ba --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql @@ -0,0 +1,285 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L320-562 + +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- CREATE TEMP VIEW v_window AS +-- SELECT i, min(i) over (order by i range between '1 day' preceding and '10 days' following) as min_i +-- FROM range(now(), now()+'100 days', '1 hour') i; + +-- RANGE offset PRECEDING/FOLLOWING tests + +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 6 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 6 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following +-- exclude current row),unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date desc range between '1 year' preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date desc range between '1 year' following and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude current row), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude group), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by salary range between 1000 preceding and 1000 following), +-- lead(salary) over(order by salary range between 1000 preceding and 1000 following), +-- nth_value(salary, 1) over(order by salary range between 1000 preceding and 1000 following), +-- salary from empsalary; + +select last(salary) over(order by salary range between 1000 preceding and 1000 following), +lag(salary) over(order by salary range between 1000 preceding and 1000 following), +salary from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by salary range between 1000 following and 3000 following +-- exclude current row), +-- lead(salary) over(order by salary range between 1000 following and 3000 following exclude ties), +-- nth_value(salary, 1) over(order by salary range between 1000 following and 3000 following +-- exclude ties), +-- salary from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select last(salary) over(order by salary range between 1000 following and 3000 following +-- exclude group), +-- lag(salary) over(order by salary range between 1000 following and 3000 following exclude group), +-- salary from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude group), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude group), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude current row), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude current row), +-- salary, enroll_date from empsalary; + +-- RANGE offset PRECEDING/FOLLOWING with null values +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following); + +-- Check overflow behavior for various integer sizes + +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x; + +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x; + +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x; + +-- Test in_range for other numeric datatypes + +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet; + +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100); +-- (8, 'infinity', 'infinity', '1000'), +-- (9, 'NaN', 'NaN', 'NaN'), +-- (0, '-infinity', '-infinity', '-1000'); -- numeric type lacks infinities + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following); + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following); + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following); -- currently unsupported + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +drop table empsalary; +drop table numerics; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/with.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql rename to sql/core/src/test/resources/sql-tests/inputs/postgreSQL/with.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index 22f3eafd6a02d..f06be5fc6aa58 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -83,7 +83,7 @@ GROUP BY t1a, t3a, t3b, t3c -ORDER BY t1a DESC, t3b DESC; +ORDER BY t1a DESC, t3b DESC, t3c ASC; -- TC 01.03 SELECT Count(DISTINCT(t1a)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql index 4f8ca8bfb27c1..bebc18a61894b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -128,7 +128,7 @@ GROUP BY t1b, HAVING t1d NOT IN (SELECT t2d FROM t2 WHERE t1d = t2d) -ORDER BY t1b DESC; +ORDER BY t1b DESC, t1d ASC; -- TC 01.05 SELECT COUNT(DISTINCT(t1a)), 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/postgreSQL/udf-aggregates_part1.sql similarity index 98% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql index d829a5c1159fd..24bc25a3fd1c5 100644 --- 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/postgreSQL/udf-aggregates_part1.sql @@ -8,7 +8,7 @@ -- 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. +-- This test file was converted from postgreSQL/aggregates_part1.sql. SELECT avg(udf(four)) AS avg_1 FROM onek; 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/postgreSQL/udf-aggregates_part2.sql similarity index 75% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql index 5636537398a86..b4054850062b7 100644 --- 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/postgreSQL/udf-aggregates_part2.sql @@ -5,7 +5,7 @@ -- 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. +-- This test file was converted from postgreSQL/aggregates_part2.sql. create temporary view int4_tbl as select * from values (0), @@ -43,42 +43,28 @@ create temporary view int4_tbl as select * from values -- -- test for bitwise integer aggregates -- --- CREATE TEMPORARY TABLE bitwise_test( --- i2 INT2, --- i4 INT4, --- i8 INT8, --- i INTEGER, --- x INT2, --- y BIT(4) --- ); +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); -- 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; +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null; + + +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test; -- -- test boolean aggregates @@ -116,50 +102,40 @@ SELECT 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); +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4); -- 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; +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0; + +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test; + +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test; + +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test; -- -- Test cases that should be optimized into indexscans instead of diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part3.sql similarity index 99% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part3.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part3.sql index 1c58620d1c11a..231c5235b313a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part3.sql @@ -5,7 +5,7 @@ -- AGGREGATES [Part 3] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 --- This test file was converted from pgSQL/aggregates_part3.sql. +-- This test file was converted from postgreSQL/aggregates_part3.sql. -- [SPARK-28865] Table inheritance -- try it on an inheritance tree diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part4.sql similarity index 99% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part4.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part4.sql index 7c7777362de8e..8aea00073eee8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part4.sql @@ -5,7 +5,7 @@ -- AGGREGATES [Part 4] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 --- This test file was converted from pgSQL/aggregates_part4.sql. +-- This test file was converted from postgreSQL/aggregates_part4.sql. -- [SPARK-27980] Ordered-Set Aggregate Functions -- ordered-set aggregates 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/postgreSQL/udf-case.sql similarity index 99% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-case.sql index 1865ee94ec1f9..8fa3c0a6dfec9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-case.sql @@ -6,7 +6,7 @@ -- 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. +-- This test file was converted from postgreSQL/case.sql. CREATE TABLE CASE_TBL ( i integer, diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql similarity index 99% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql index c05aa156a13bf..e6fe1078b0d24 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql @@ -6,7 +6,7 @@ -- Test JOIN clauses -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql -- --- This test file was converted from pgSQL/join.sql. +-- This test file was converted from postgreSQL/join.sql. CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-select_having.sql similarity index 96% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-select_having.sql index c8e4346cedb89..412d45b49a184 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-select_having.sql @@ -5,7 +5,7 @@ -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql -- --- This test file was converted from inputs/pgSQL/select_having.sql +-- This test file was converted from inputs/postgreSQL/select_having.sql -- TODO: We should add UDFs in GROUP BY clause when [SPARK-28386] and [SPARK-26741] is resolved. -- load test data diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_implicit.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-select_implicit.sql similarity index 98% rename from sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_implicit.sql rename to sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-select_implicit.sql index 373896ccd1674..1cbd77c6cf86d 100755 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_implicit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-select_implicit.sql @@ -9,7 +9,7 @@ -- - thomas 1998-07-09 -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_implicit.sql -- --- This test file was converted from pgSQL/select_implicit.sql +-- This test file was converted from postgreSQL/select_implicit.sql -- load test data CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out new file mode 100644 index 0000000000000..7cbd26e87bd2b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out @@ -0,0 +1,164 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 20 + + +-- !query 0 +select bit_count(null) +-- !query 0 schema +struct +-- !query 0 output +NULL + + +-- !query 1 +select bit_count(true) +-- !query 1 schema +struct +-- !query 1 output +1 + + +-- !query 2 +select bit_count(false) +-- !query 2 schema +struct +-- !query 2 output +0 + + +-- !query 3 +select bit_count(cast(1 as tinyint)) +-- !query 3 schema +struct +-- !query 3 output +1 + + +-- !query 4 +select bit_count(cast(2 as tinyint)) +-- !query 4 schema +struct +-- !query 4 output +1 + + +-- !query 5 +select bit_count(cast(3 as tinyint)) +-- !query 5 schema +struct +-- !query 5 output +2 + + +-- !query 6 +select bit_count(1S) +-- !query 6 schema +struct +-- !query 6 output +1 + + +-- !query 7 +select bit_count(2S) +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +select bit_count(3S) +-- !query 8 schema +struct +-- !query 8 output +2 + + +-- !query 9 +select bit_count(1) +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +select bit_count(2) +-- !query 10 schema +struct +-- !query 10 output +1 + + +-- !query 11 +select bit_count(3) +-- !query 11 schema +struct +-- !query 11 output +2 + + +-- !query 12 +select bit_count(1L) +-- !query 12 schema +struct +-- !query 12 output +1 + + +-- !query 13 +select bit_count(2L) +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +select bit_count(3L) +-- !query 14 schema +struct +-- !query 14 output +2 + + +-- !query 15 +select bit_count(-1L) +-- !query 15 schema +struct +-- !query 15 output +64 + + +-- !query 16 +select bit_count(9223372036854775807L) +-- !query 16 schema +struct +-- !query 16 output +63 + + +-- !query 17 +select bit_count(-9223372036854775808L) +-- !query 17 schema +struct +-- !query 17 output +1 + + +-- !query 18 +select bit_count("bit count") +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +cannot resolve 'bit_count('bit count')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''bit count'' is of string type.; line 1 pos 7 + + +-- !query 19 +select bit_count('a') +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +cannot resolve 'bit_count('a')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''a'' is of string type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 776786850e9da..8f4edf1960755 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 52 +-- Number of queries: 110 -- !query 0 @@ -357,41 +357,41 @@ struct -- !query 44 select date_part('second', c) from t -- !query 44 schema -struct +struct -- !query 44 output -9 +9.123456 -- !query 45 select date_part('s', c) from t -- !query 45 schema -struct +struct -- !query 45 output -9 +9.123456 -- !query 46 select date_part('sec', c) from t -- !query 46 schema -struct +struct -- !query 46 output -9 +9.123456 -- !query 47 select date_part('seconds', c) from t -- !query 47 schema -struct +struct -- !query 47 output -9 +9.123456 -- !query 48 select date_part('secs', c) from t -- !query 48 schema -struct +struct -- !query 48 output -9 +9.123456 -- !query 49 @@ -400,7 +400,7 @@ select date_part('not_supported', c) from t struct<> -- !query 49 output org.apache.spark.sql.AnalysisException -Literals of type 'not_supported' are currently not supported.;; line 1 pos 7 +Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7 -- !query 50 @@ -418,3 +418,469 @@ select date_part(null, c) from t struct -- !query 51 output NULL + + +-- !query 52 +CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +select date_part('millennium', c) from t2 +-- !query 53 schema +struct +-- !query 53 output +1 + + +-- !query 54 +select date_part('millennia', c) from t2 +-- !query 54 schema +struct +-- !query 54 output +1 + + +-- !query 55 +select date_part('mil', c) from t2 +-- !query 55 schema +struct +-- !query 55 output +1 + + +-- !query 56 +select date_part('mils', c) from t2 +-- !query 56 schema +struct +-- !query 56 output +1 + + +-- !query 57 +select date_part('century', c) from t2 +-- !query 57 schema +struct +-- !query 57 output +10 + + +-- !query 58 +select date_part('centuries', c) from t2 +-- !query 58 schema +struct +-- !query 58 output +10 + + +-- !query 59 +select date_part('c', c) from t2 +-- !query 59 schema +struct +-- !query 59 output +10 + + +-- !query 60 +select date_part('cent', c) from t2 +-- !query 60 schema +struct +-- !query 60 output +10 + + +-- !query 61 +select date_part('decade', c) from t2 +-- !query 61 schema +struct +-- !query 61 output +101 + + +-- !query 62 +select date_part('decades', c) from t2 +-- !query 62 schema +struct +-- !query 62 output +101 + + +-- !query 63 +select date_part('dec', c) from t2 +-- !query 63 schema +struct +-- !query 63 output +101 + + +-- !query 64 +select date_part('decs', c) from t2 +-- !query 64 schema +struct +-- !query 64 output +101 + + +-- !query 65 +select date_part('year', c) from t2 +-- !query 65 schema +struct +-- !query 65 output +1010 + + +-- !query 66 +select date_part('y', c) from t2 +-- !query 66 schema +struct +-- !query 66 output +1010 + + +-- !query 67 +select date_part('years', c) from t2 +-- !query 67 schema +struct +-- !query 67 output +1010 + + +-- !query 68 +select date_part('yr', c) from t2 +-- !query 68 schema +struct +-- !query 68 output +1010 + + +-- !query 69 +select date_part('yrs', c) from t2 +-- !query 69 schema +struct +-- !query 69 output +1010 + + +-- !query 70 +select date_part('quarter', c) from t2 +-- !query 70 schema +struct +-- !query 70 output +4 + + +-- !query 71 +select date_part('qtr', c) from t2 +-- !query 71 schema +struct +-- !query 71 output +4 + + +-- !query 72 +select date_part('month', c) from t2 +-- !query 72 schema +struct +-- !query 72 output +9 + + +-- !query 73 +select date_part('mon', c) from t2 +-- !query 73 schema +struct +-- !query 73 output +9 + + +-- !query 74 +select date_part('mons', c) from t2 +-- !query 74 schema +struct +-- !query 74 output +9 + + +-- !query 75 +select date_part('months', c) from t2 +-- !query 75 schema +struct +-- !query 75 output +9 + + +-- !query 76 +select date_part('day', c) from t2 +-- !query 76 schema +struct +-- !query 76 output +8 + + +-- !query 77 +select date_part('d', c) from t2 +-- !query 77 schema +struct +-- !query 77 output +8 + + +-- !query 78 +select date_part('days', c) from t2 +-- !query 78 schema +struct +-- !query 78 output +8 + + +-- !query 79 +select date_part('hour', c) from t2 +-- !query 79 schema +struct +-- !query 79 output +7 + + +-- !query 80 +select date_part('h', c) from t2 +-- !query 80 schema +struct +-- !query 80 output +7 + + +-- !query 81 +select date_part('hours', c) from t2 +-- !query 81 schema +struct +-- !query 81 output +7 + + +-- !query 82 +select date_part('hr', c) from t2 +-- !query 82 schema +struct +-- !query 82 output +7 + + +-- !query 83 +select date_part('hrs', c) from t2 +-- !query 83 schema +struct +-- !query 83 output +7 + + +-- !query 84 +select date_part('minute', c) from t2 +-- !query 84 schema +struct +-- !query 84 output +6 + + +-- !query 85 +select date_part('m', c) from t2 +-- !query 85 schema +struct +-- !query 85 output +6 + + +-- !query 86 +select date_part('min', c) from t2 +-- !query 86 schema +struct +-- !query 86 output +6 + + +-- !query 87 +select date_part('mins', c) from t2 +-- !query 87 schema +struct +-- !query 87 output +6 + + +-- !query 88 +select date_part('minutes', c) from t2 +-- !query 88 schema +struct +-- !query 88 output +6 + + +-- !query 89 +select date_part('second', c) from t2 +-- !query 89 schema +struct +-- !query 89 output +5.004003 + + +-- !query 90 +select date_part('s', c) from t2 +-- !query 90 schema +struct +-- !query 90 output +5.004003 + + +-- !query 91 +select date_part('sec', c) from t2 +-- !query 91 schema +struct +-- !query 91 output +5.004003 + + +-- !query 92 +select date_part('seconds', c) from t2 +-- !query 92 schema +struct +-- !query 92 output +5.004003 + + +-- !query 93 +select date_part('secs', c) from t2 +-- !query 93 schema +struct +-- !query 93 output +5.004003 + + +-- !query 94 +select date_part('milliseconds', c) from t2 +-- !query 94 schema +struct +-- !query 94 output +5004.003 + + +-- !query 95 +select date_part('msec', c) from t2 +-- !query 95 schema +struct +-- !query 95 output +5004.003 + + +-- !query 96 +select date_part('msecs', c) from t2 +-- !query 96 schema +struct +-- !query 96 output +5004.003 + + +-- !query 97 +select date_part('millisecon', c) from t2 +-- !query 97 schema +struct +-- !query 97 output +5004.003 + + +-- !query 98 +select date_part('mseconds', c) from t2 +-- !query 98 schema +struct +-- !query 98 output +5004.003 + + +-- !query 99 +select date_part('ms', c) from t2 +-- !query 99 schema +struct +-- !query 99 output +5004.003 + + +-- !query 100 +select date_part('microseconds', c) from t2 +-- !query 100 schema +struct +-- !query 100 output +5004003 + + +-- !query 101 +select date_part('usec', c) from t2 +-- !query 101 schema +struct +-- !query 101 output +5004003 + + +-- !query 102 +select date_part('usecs', c) from t2 +-- !query 102 schema +struct +-- !query 102 output +5004003 + + +-- !query 103 +select date_part('useconds', c) from t2 +-- !query 103 schema +struct +-- !query 103 output +5004003 + + +-- !query 104 +select date_part('microsecon', c) from t2 +-- !query 104 schema +struct +-- !query 104 output +5004003 + + +-- !query 105 +select date_part('us', c) from t2 +-- !query 105 schema +struct +-- !query 105 output +5004003 + + +-- !query 106 +select date_part('epoch', c) from t2 +-- !query 106 schema +struct +-- !query 106 output +31873892788.332003 + + +-- !query 107 +select date_part('not_supported', c) from t2 +-- !query 107 schema +struct<> +-- !query 107 output +org.apache.spark.sql.AnalysisException +Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7 + + +-- !query 108 +select date_part(c, c) from t2 +-- !query 108 schema +struct<> +-- !query 108 output +org.apache.spark.sql.AnalysisException +The field parameter needs to be a foldable string value.;; line 1 pos 7 + + +-- !query 109 +select date_part(null, c) from t2 +-- !query 109 schema +struct +-- !query 109 output +NULL 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 178400e5706b8..0f4036cad6125 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: 15 +-- Number of queries: 17 -- !query 0 @@ -126,6 +126,22 @@ struct -- !query 14 select date '2001-10-01' - date '2001-09-28' -- !query 14 schema -struct +struct -- !query 14 output -3 +interval 3 days + + +-- !query 15 +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query 15 schema +struct +-- !query 15 output +interval 12 weeks 2 days 14 hours 48 minutes 47 seconds 654 milliseconds 322 microseconds + + +-- !query 16 +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query 16 schema +struct +-- !query 16 output +interval -12 weeks -2 days -14 hours -48 minutes -47 seconds -654 milliseconds -322 microseconds 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 17dd317f63b70..e12889f9bff83 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 @@ -56,12 +56,12 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 5 @@ -90,13 +90,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 7 @@ -125,13 +125,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 9 @@ -152,13 +152,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=11] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 4 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 10 @@ -187,13 +187,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 12 @@ -214,13 +214,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=11] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 4 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 13 @@ -241,13 +241,13 @@ hr int Database default Table t Partition Values [ds=2017-09-01, hr=5] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-09-01/hr=5 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-09-01/hr=5 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 2 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 14 diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out index 7873085da5069..ffd6fb31620d3 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out @@ -29,7 +29,7 @@ Created By [not included in comparison] Type MANAGED Provider parquet Comment added -Location [not included in comparison]sql/core/spark-warehouse/table_with_comment +Location [not included in comparison]/{warehouse_dir}/table_with_comment -- !query 2 @@ -60,7 +60,7 @@ Type MANAGED Provider parquet Comment modified comment Table Properties [type=parquet] -Location [not included in comparison]sql/core/spark-warehouse/table_with_comment +Location [not included in comparison]/{warehouse_dir}/table_with_comment -- !query 4 @@ -95,7 +95,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type MANAGED Provider parquet -Location [not included in comparison]sql/core/spark-warehouse/table_comment +Location [not included in comparison]/{warehouse_dir}/table_comment -- !query 7 @@ -123,7 +123,7 @@ Created By [not included in comparison] Type MANAGED Provider parquet Comment added comment -Location [not included in comparison]sql/core/spark-warehouse/table_comment +Location [not included in comparison]/{warehouse_dir}/table_comment -- !query 9 @@ -150,7 +150,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type MANAGED Provider parquet -Location [not included in comparison]sql/core/spark-warehouse/table_comment +Location [not included in comparison]/{warehouse_dir}/table_comment -- !query 11 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 f58bdb5446b64..0cb442a6d3496 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 @@ -131,7 +131,7 @@ Bucket Columns [`a`] Sort Columns [`b`] Comment table_comment Table Properties [t=test, e=3] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -163,7 +163,7 @@ Bucket Columns [`a`] Sort Columns [`b`] Comment table_comment Table Properties [t=test, e=3] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -203,7 +203,7 @@ Bucket Columns [`a`] Sort Columns [`b`] Comment table_comment Table Properties [t=test] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -242,7 +242,7 @@ Num Buckets 2 Bucket Columns [`a`] Sort Columns [`b`] Table Properties [t=test] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -280,7 +280,7 @@ d string Database default Table t Partition Values [c=Us, d=1] -Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1 +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 Storage Properties [a=1, b=2] Created Time [not included in comparison] Last Access [not included in comparison] @@ -289,7 +289,7 @@ Last Access [not included in comparison] Num Buckets 2 Bucket Columns [`a`] Sort Columns [`b`] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] @@ -311,7 +311,7 @@ d string Database default Table t Partition Values [c=Us, d=1] -Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1 +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 Storage Properties [a=1, b=2] Created Time [not included in comparison] Last Access [not included in comparison] @@ -320,7 +320,7 @@ Last Access [not included in comparison] Num Buckets 2 Bucket Columns [`a`] Sort Columns [`b`] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 4a08cfada292d..85c938773efec 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -58,6 +58,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -110,6 +114,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -165,6 +173,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -179,6 +191,10 @@ Input : [key#x, val#x] (5) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] @@ -227,6 +243,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] @@ -241,6 +261,10 @@ Input : [key#x, val#x] (5) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -284,12 +308,19 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] (3) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct (4) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -334,6 +365,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), IsNotNull(val), GreaterThan(val,3)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -360,6 +395,10 @@ Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (5) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(val,2)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -393,6 +432,10 @@ Subquery:2 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.explain_temp3 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct (13) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -437,6 +480,9 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -459,6 +505,10 @@ Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (4) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -492,6 +542,10 @@ Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (11) Scan parquet default.explain_temp3 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -529,6 +583,9 @@ struct (1) Scan parquet default.explain_temp1 Output: [] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct<> (2) ColumnarToRow [codegen id : 1] Input: [] @@ -549,6 +606,9 @@ Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (4) Scan parquet default.explain_temp1 Output: [key#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input: [key#x] @@ -591,6 +651,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] @@ -605,6 +669,10 @@ Input : [key#x, val#x] (5) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -654,6 +722,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index e007fa8368ef5..1424cc45fbb42 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -365,41 +365,41 @@ struct -- !query 45 select extract(second from c) from t -- !query 45 schema -struct +struct -- !query 45 output -9 +9.123456 -- !query 46 select extract(s from c) from t -- !query 46 schema -struct +struct -- !query 46 output -9 +9.123456 -- !query 47 select extract(sec from c) from t -- !query 47 schema -struct +struct -- !query 47 output -9 +9.123456 -- !query 48 select extract(seconds from c) from t -- !query 48 schema -struct +struct -- !query 48 output -9 +9.123456 -- !query 49 select extract(secs from c) from t -- !query 49 schema -struct +struct -- !query 49 output -9 +9.123456 -- !query 50 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 3a5df254f2cd9..545aa238dd756 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: 52 +-- Number of queries: 56 -- !query 0 @@ -291,39 +291,39 @@ struct<> -- !query 31 -SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0 +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output -NULL NULL NULL +NULL NULL NULL NULL NULL -- !query 32 -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4 +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output -NULL NULL NULL +NULL NULL NULL NULL NULL -- !query 33 -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5 +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output -false true true +false true true false true -- !query 34 -SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k -- !query 34 schema -struct +struct -- !query 34 output -1 false true true -2 true true true -3 false false false -4 NULL NULL NULL -5 false true true +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true -- !query 35 @@ -389,7 +389,7 @@ SELECT some(1S) 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 +cannot resolve 'any(1S)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -411,10 +411,28 @@ cannot resolve 'every('true')' due to data type mismatch: Input to function 'eve -- !query 43 -SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +SELECT bool_and(1.0) -- !query 43 schema -struct +struct<> -- !query 43 output +org.apache.spark.sql.AnalysisException +cannot resolve 'every(1.0BD)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [decimal(2,1)].; line 1 pos 7 + + +-- !query 44 +SELECT bool_or(1.0D) +-- !query 44 schema +struct<> +-- !query 44 output +org.apache.spark.sql.AnalysisException +cannot resolve 'any(1.0D)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [double].; line 1 pos 7 + + +-- !query 45 +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 45 schema +struct +-- !query 45 output 1 false false 1 true false 2 true true @@ -427,11 +445,11 @@ struct --- !query 44 output +-- !query 46 schema +struct +-- !query 46 output 1 false false 1 true true 2 true true @@ -444,11 +462,11 @@ struct --- !query 45 output +-- !query 47 output 1 false false 1 true true 2 true true @@ -461,37 +479,71 @@ struct +-- !query 48 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 49 +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 49 schema +struct +-- !query 49 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 50 SELECT count(*) FROM test_agg HAVING count(*) > 1L --- !query 46 schema +-- !query 50 schema struct --- !query 46 output +-- !query 50 output 10 --- !query 47 +-- !query 51 SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true --- !query 47 schema +-- !query 51 schema struct --- !query 47 output +-- !query 51 output 1 true 2 true 5 true --- !query 48 +-- !query 52 SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L --- !query 48 schema +-- !query 52 schema struct --- !query 48 output +-- !query 52 output 10 --- !query 49 +-- !query 53 SELECT count(*) FROM test_agg WHERE count(*) > 1L --- !query 49 schema +-- !query 53 schema struct<> --- !query 49 output +-- !query 53 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -499,11 +551,11 @@ Expression in where clause: [(count(1) > 1L)] Invalid expressions: [count(1)]; --- !query 50 +-- !query 54 SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L --- !query 50 schema +-- !query 54 schema struct<> --- !query 50 output +-- !query 54 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -511,11 +563,11 @@ Expression in where clause: [((count(1) + 1L) > 1L)] Invalid expressions: [count(1)]; --- !query 51 +-- !query 55 SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 --- !query 51 schema +-- !query 55 schema struct<> --- !query 51 output +-- !query 55 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. 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 e1e8d685e8787..fd6e51b2385de 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 46 +-- Number of queries: 59 -- !query 0 @@ -337,10 +337,114 @@ interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseco -- !query 36 -select interval 10 nanoseconds +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 36 schema -struct<> +struct -- !query 36 output +interval 32 years 1 months -14 weeks -6 hours -35 minutes -110 milliseconds -13 microseconds + + +-- !query 37 +select interval '0 0:0:0.1' day to second +-- !query 37 schema +struct +-- !query 37 output +interval 100 milliseconds + + +-- !query 38 +select interval '10-9' year to month +-- !query 38 schema +struct +-- !query 38 output +interval 10 years 9 months + + +-- !query 39 +select interval '20 15:40:32.99899999' day to hour +-- !query 39 schema +struct +-- !query 39 output +interval 2 weeks 6 days 15 hours + + +-- !query 40 +select interval '20 15:40:32.99899999' day to minute +-- !query 40 schema +struct +-- !query 40 output +interval 2 weeks 6 days 15 hours 40 minutes + + +-- !query 41 +select interval '20 15:40:32.99899999' day to second +-- !query 41 schema +struct +-- !query 41 output +interval 2 weeks 6 days 15 hours 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 42 +select interval '15:40:32.99899999' hour to minute +-- !query 42 schema +struct +-- !query 42 output +interval 15 hours 40 minutes + + +-- !query 43 +select interval '15:40.99899999' hour to second +-- !query 43 schema +struct +-- !query 43 output +interval 15 minutes 40 seconds 998 milliseconds 999 microseconds + + +-- !query 44 +select interval '15:40' hour to second +-- !query 44 schema +struct +-- !query 44 output +interval 15 hours 40 minutes + + +-- !query 45 +select interval '15:40:32.99899999' hour to second +-- !query 45 schema +struct +-- !query 45 output +interval 15 hours 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 46 +select interval '20 40:32.99899999' minute to second +-- !query 46 schema +struct +-- !query 46 output +interval 2 weeks 6 days 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 47 +select interval '40:32.99899999' minute to second +-- !query 47 schema +struct +-- !query 47 output +interval 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 48 +select interval '40:32' minute to second +-- !query 48 schema +struct +-- !query 48 output +interval 40 minutes 32 seconds + + +-- !query 49 +select interval 10 nanoseconds +-- !query 49 schema +struct<> +-- !query 49 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'interval 10 nanoseconds'(line 1, pos 19) @@ -350,11 +454,11 @@ select interval 10 nanoseconds -------------------^^^ --- !query 37 +-- !query 50 select GEO '(10,-6)' --- !query 37 schema +-- !query 50 schema struct<> --- !query 37 output +-- !query 50 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'GEO' are currently not supported.(line 1, pos 7) @@ -364,19 +468,19 @@ select GEO '(10,-6)' -------^^^ --- !query 38 +-- !query 51 select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD --- !query 38 schema +-- !query 51 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> --- !query 38 output +-- !query 51 output 90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 --- !query 39 +-- !query 52 select 1.20E-38BD --- !query 39 schema +-- !query 52 schema struct<> --- !query 39 output +-- !query 52 output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38(line 1, pos 7) @@ -386,19 +490,19 @@ select 1.20E-38BD -------^^^ --- !query 40 +-- !query 53 select x'2379ACFe' --- !query 40 schema +-- !query 53 schema struct --- !query 40 output +-- !query 53 output #y�� --- !query 41 +-- !query 54 select X'XuZ' --- !query 41 schema +-- !query 54 schema struct<> --- !query 41 output +-- !query 54 output org.apache.spark.sql.catalyst.parser.ParseException contains illegal character for hexBinary: 0XuZ(line 1, pos 7) @@ -408,33 +512,33 @@ select X'XuZ' -------^^^ --- !query 42 +-- !query 55 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 --- !query 42 schema +-- !query 55 schema struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> --- !query 42 output +-- !query 55 output 3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 --- !query 43 +-- !query 56 select map(1, interval 1 day, 2, interval 3 week) --- !query 43 schema +-- !query 56 schema struct> --- !query 43 output +-- !query 56 output {1:interval 1 days,2:interval 3 weeks} --- !query 44 +-- !query 57 select interval 'interval 3 year 1 hour' --- !query 44 schema +-- !query 57 schema struct --- !query 44 output +-- !query 57 output interval 3 years 1 hours --- !query 45 +-- !query 58 select interval '3 year 1 hour' --- !query 45 schema -struct --- !query 45 output -NULL +-- !query 58 schema +struct +-- !query 58 output +interval 3 years 1 hours 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 deleted file mode 100644 index 2b5371a657196..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out +++ /dev/null @@ -1,156 +0,0 @@ --- 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 - (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`, - (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 - (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`, - (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/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out new file mode 100644 index 0000000000000..fdca51ff1325b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out @@ -0,0 +1,303 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 29 + + +-- !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 +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query 2 schema +struct +-- !query 2 output +NULL NULL + + +-- !query 3 +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query 3 schema +struct +-- !query 3 output +NULL NULL + + +-- !query 4 +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query 4 schema +struct +-- !query 4 output +1 1 1 1 7 7 7 3 + + +-- !query 5 +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1 +-- !query 5 schema +struct +-- !query 5 output +1 1 1 +3 3 NULL +7 7 3 + + +-- !query 6 +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query 6 schema +struct +-- !query 6 output +1 1 +3 3 + + +-- !query 7 +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query 7 schema +struct +-- !query 7 output +1 1 1 +3 3 3 +7 7 7 + + +-- !query 8 +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query 8 schema +struct +-- !query 8 output +1 1 1 +3 3 3 +7 7 7 + + +-- !query 9 +SELECT + (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`, + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query 9 schema +struct +-- !query 9 output +true true false true false true true true true + + +-- !query 10 +SELECT + (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`, + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query 10 schema +struct +-- !query 10 output +true false true false true true true true true + + +-- !query 11 +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 +-- !query 12 schema +struct +-- !query 12 output +NULL NULL + + +-- !query 13 +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test +-- !query 13 schema +struct +-- !query 13 output +false true false NULL false true + + +-- !query 14 +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test +-- !query 14 schema +struct +-- !query 14 output +false true false NULL false true + + +-- !query 15 +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test +-- !query 15 schema +struct +-- !query 15 output +true true false NULL false true + + +-- !query 16 +select min(unique1) from tenk1 +-- !query 16 schema +struct +-- !query 16 output +0 + + +-- !query 17 +select max(unique1) from tenk1 +-- !query 17 schema +struct +-- !query 17 output +9999 + + +-- !query 18 +select max(unique1) from tenk1 where unique1 < 42 +-- !query 18 schema +struct +-- !query 18 output +41 + + +-- !query 19 +select max(unique1) from tenk1 where unique1 > 42 +-- !query 19 schema +struct +-- !query 19 output +9999 + + +-- !query 20 +select max(unique1) from tenk1 where unique1 > 42000 +-- !query 20 schema +struct +-- !query 20 output +NULL + + +-- !query 21 +select max(tenthous) from tenk1 where thousand = 33 +-- !query 21 schema +struct +-- !query 21 output +9033 + + +-- !query 22 +select min(tenthous) from tenk1 where thousand = 33 +-- !query 22 schema +struct +-- !query 22 output +33 + + +-- !query 23 +select distinct max(unique2) from tenk1 +-- !query 23 schema +struct +-- !query 23 output +9999 + + +-- !query 24 +select max(unique2) from tenk1 order by 1 +-- !query 24 schema +struct +-- !query 24 output +9999 + + +-- !query 25 +select max(unique2) from tenk1 order by max(unique2) +-- !query 25 schema +struct +-- !query 25 output +9999 + + +-- !query 26 +select max(unique2) from tenk1 order by max(unique2)+1 +-- !query 26 schema +struct +-- !query 26 output +9999 + + +-- !query 27 +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 27 schema +struct +-- !query 27 output +9999 3 +9999 2 +9999 1 + + +-- !query 28 +select max(100) from tenk1 +-- !query 28 schema +struct +-- !query 28 output +100 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part3.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part4.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part4.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part4.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out similarity index 93% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out index fbd32fc909076..5e8383e794579 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out @@ -42,7 +42,7 @@ struct<> -- !query 4 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* This is an example of SQL which should not execute: @@ -58,7 +58,7 @@ struct<> -- !query 5 output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -74,7 +74,7 @@ struct<> -- !query 6 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* @@ -92,7 +92,7 @@ struct<> -- !query 7 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -114,7 +114,7 @@ struct<> -- !query 8 output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -134,7 +134,7 @@ struct<> -- !query 9 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* Second level of nesting... @@ -150,7 +150,7 @@ struct<> -- !query 10 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* Third level of nesting... @@ -170,7 +170,7 @@ struct<> -- !query 11 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -189,7 +189,7 @@ struct<> -- !query 12 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 37) +mismatched input '' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 37) == SQL == /* and this is the end of the file */ diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out new file mode 100644 index 0000000000000..bed5d7a56c1f8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -0,0 +1,194 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 24 + + +-- !query 0 +SELECT interval '999' second +-- !query 0 schema +struct +-- !query 0 output +interval 16 minutes 39 seconds + + +-- !query 1 +SELECT interval '999' minute +-- !query 1 schema +struct +-- !query 1 output +interval 16 hours 39 minutes + + +-- !query 2 +SELECT interval '999' hour +-- !query 2 schema +struct +-- !query 2 output +interval 5 weeks 6 days 15 hours + + +-- !query 3 +SELECT interval '999' day +-- !query 3 schema +struct +-- !query 3 output +interval 142 weeks 5 days + + +-- !query 4 +SELECT interval '999' month +-- !query 4 schema +struct +-- !query 4 output +interval 83 years 3 months + + +-- !query 5 +SELECT interval '1' year +-- !query 5 schema +struct +-- !query 5 output +interval 1 years + + +-- !query 6 +SELECT interval '2' month +-- !query 6 schema +struct +-- !query 6 output +interval 2 months + + +-- !query 7 +SELECT interval '3' day +-- !query 7 schema +struct +-- !query 7 output +interval 3 days + + +-- !query 8 +SELECT interval '4' hour +-- !query 8 schema +struct +-- !query 8 output +interval 4 hours + + +-- !query 9 +SELECT interval '5' minute +-- !query 9 schema +struct +-- !query 9 output +interval 5 minutes + + +-- !query 10 +SELECT interval '6' second +-- !query 10 schema +struct +-- !query 10 output +interval 6 seconds + + +-- !query 11 +SELECT interval '1-2' year to month +-- !query 11 schema +struct +-- !query 11 output +interval 1 years 2 months + + +-- !query 12 +SELECT interval '1 2:03' day to hour +-- !query 12 schema +struct +-- !query 12 output +interval 1 days 2 hours + + +-- !query 13 +SELECT interval '1 2:03:04' day to hour +-- !query 13 schema +struct +-- !query 13 output +interval 1 days 2 hours + + +-- !query 14 +SELECT interval '1 2:03' day to minute +-- !query 14 schema +struct +-- !query 14 output +interval 1 days 2 hours 3 minutes + + +-- !query 15 +SELECT interval '1 2:03:04' day to minute +-- !query 15 schema +struct +-- !query 15 output +interval 1 days 2 hours 3 minutes + + +-- !query 16 +SELECT interval '1 2:03' day to second +-- !query 16 schema +struct +-- !query 16 output +interval 1 days 2 hours 3 minutes + + +-- !query 17 +SELECT interval '1 2:03:04' day to second +-- !query 17 schema +struct +-- !query 17 output +interval 1 days 2 hours 3 minutes 4 seconds + + +-- !query 18 +SELECT interval '1 2:03' hour to minute +-- !query 18 schema +struct +-- !query 18 output +interval 1 days 2 hours 3 minutes + + +-- !query 19 +SELECT interval '1 2:03:04' hour to minute +-- !query 19 schema +struct +-- !query 19 output +interval 1 days 2 hours 3 minutes + + +-- !query 20 +SELECT interval '1 2:03' hour to second +-- !query 20 schema +struct +-- !query 20 output +interval 1 days 2 hours 3 minutes + + +-- !query 21 +SELECT interval '1 2:03:04' hour to second +-- !query 21 schema +struct +-- !query 21 output +interval 1 days 2 hours 3 minutes 4 seconds + + +-- !query 22 +SELECT interval '1 2:03' minute to second +-- !query 22 schema +struct +-- !query 22 output +interval 1 days 2 minutes 3 seconds + + +-- !query 23 +SELECT interval '1 2:03:04' minute to second +-- !query 23 schema +struct +-- !query 23 output +interval 1 days 2 hours 3 minutes 4 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out similarity index 99% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out index f75fe0519645b..f823d4e1459d4 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 185 +-- Number of queries: 181 -- !query 0 @@ -2536,7 +2536,9 @@ struct -- !query 107 -DROP TABLE IF EXISTS tt3 +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2) -- !query 107 schema struct<> -- !query 107 output @@ -2544,7 +2546,9 @@ struct<> -- !query 108 -CREATE TABLE tt3(f1 int, f2 string) USING parquet +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1) -- !query 108 schema struct<> -- !query 108 output @@ -2552,38 +2556,6 @@ struct<> -- !query 109 -INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x --- !query 109 schema -struct<> --- !query 109 output - - - --- !query 110 -DROP TABLE IF EXISTS tt4 --- !query 110 schema -struct<> --- !query 110 output - - - --- !query 111 -CREATE TABLE tt4(f1 int) USING parquet --- !query 111 schema -struct<> --- !query 111 output - - - --- !query 112 -INSERT INTO tt4 VALUES (0),(1),(9999) --- !query 112 schema -struct<> --- !query 112 output - - - --- !query 113 SELECT a.f1 FROM tt4 a LEFT JOIN ( @@ -2592,242 +2564,242 @@ LEFT JOIN ( WHERE c.f1 IS NULL ) AS d ON (a.f1 = d.f1) WHERE d.f1 IS NULL --- !query 113 schema +-- !query 109 schema struct --- !query 113 output +-- !query 109 output 0 1 9999 --- !query 114 +-- !query 110 create or replace temporary view tt5 as select * from (values (1, 10), (1, 11)) as v(f1, f2) --- !query 114 schema +-- !query 110 schema struct<> --- !query 114 output +-- !query 110 output --- !query 115 +-- !query 111 create or replace temporary view tt6 as select * from (values (1, 9), (1, 2), (2, 9)) as v(f1, f2) --- !query 115 schema +-- !query 111 schema struct<> --- !query 115 output +-- !query 111 output --- !query 116 +-- !query 112 select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2 --- !query 116 schema +-- !query 112 schema struct --- !query 116 output +-- !query 112 output 1 10 1 9 --- !query 117 +-- !query 113 create or replace temporary view xx as select * from (values (1), (2), (3)) as v(pkxx) --- !query 117 schema +-- !query 113 schema struct<> --- !query 117 output +-- !query 113 output --- !query 118 +-- !query 114 create or replace temporary view yy as select * from (values (101, 1), (201, 2), (301, NULL)) as v(pkyy, pkxx) --- !query 118 schema +-- !query 114 schema struct<> --- !query 118 output +-- !query 114 output --- !query 119 +-- !query 115 select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx from yy left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy left join xx xxa on yya.pkxx = xxa.pkxx left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx --- !query 119 schema +-- !query 115 schema struct --- !query 119 output +-- !query 115 output 101 1 101 1 1 201 2 NULL NULL 1 301 NULL NULL NULL 1 --- !query 120 +-- !query 116 create or replace temporary view zt1 as select * from (values (53)) as v(f1) --- !query 120 schema +-- !query 116 schema struct<> --- !query 120 output +-- !query 116 output --- !query 121 +-- !query 117 create or replace temporary view zt2 as select * from (values (53)) as v(f2) --- !query 121 schema +-- !query 117 schema struct<> --- !query 121 output +-- !query 117 output --- !query 122 +-- !query 118 create or replace temporary view zt3(f3 int) using parquet --- !query 122 schema +-- !query 118 schema struct<> --- !query 122 output +-- !query 118 output --- !query 123 +-- !query 119 select * from zt2 left join zt3 on (f2 = f3) left join zt1 on (f3 = f1) where f2 = 53 --- !query 123 schema +-- !query 119 schema struct --- !query 123 output +-- !query 119 output 53 NULL NULL --- !query 124 +-- !query 120 create temp view zv1 as select *,'dummy' AS junk from zt1 --- !query 124 schema +-- !query 120 schema struct<> --- !query 124 output +-- !query 120 output --- !query 125 +-- !query 121 select * from zt2 left join zt3 on (f2 = f3) left join zv1 on (f3 = f1) where f2 = 53 --- !query 125 schema +-- !query 121 schema struct --- !query 125 output +-- !query 121 output 53 NULL NULL NULL --- !query 126 +-- !query 122 select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred from tenk1 a left join tenk1 b on a.unique2 = b.tenthous where a.unique1 = 42 and ((b.unique2 is null and a.ten = 2) or b.hundred = 3) --- !query 126 schema +-- !query 122 schema struct --- !query 126 output +-- !query 122 output --- !query 127 +-- !query 123 create or replace temporary view a (i integer) using parquet --- !query 127 schema +-- !query 123 schema struct<> --- !query 127 output +-- !query 123 output --- !query 128 +-- !query 124 create or replace temporary view b (x integer, y integer) using parquet --- !query 128 schema +-- !query 124 schema struct<> --- !query 128 output +-- !query 124 output --- !query 129 +-- !query 125 select * from a left join b on i = x and i = y and x = i --- !query 129 schema +-- !query 125 schema struct --- !query 129 output +-- !query 125 output --- !query 130 +-- !query 126 select t1.q2, count(t2.*) from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 130 schema +-- !query 126 schema struct --- !query 130 output +-- !query 126 output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 131 +-- !query 127 select t1.q2, count(t2.*) from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 131 schema +-- !query 127 schema struct --- !query 131 output +-- !query 127 output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 132 +-- !query 128 select t1.q2, count(t2.*) from int8_tbl t1 left join (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 132 schema +-- !query 128 schema struct --- !query 132 output +-- !query 128 output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 133 +-- !query 129 create or replace temporary view a as select * from (values ('p'), ('q')) as v(code) --- !query 133 schema +-- !query 129 schema struct<> --- !query 133 output +-- !query 129 output --- !query 134 +-- !query 130 create or replace temporary view b as select * from (values ('p', 1), ('p', 2)) as v(a, num) --- !query 134 schema +-- !query 130 schema struct<> --- !query 134 output +-- !query 130 output --- !query 135 +-- !query 131 create or replace temporary view c as select * from (values ('A', 'p'), ('B', 'q'), ('C', null)) as v(name, a) --- !query 135 schema +-- !query 131 schema struct<> --- !query 135 output +-- !query 131 output --- !query 136 +-- !query 132 select c.name, ss.code, ss.b_cnt, ss.const from c left join (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const @@ -2837,15 +2809,15 @@ from c left join ) as ss on (c.a = ss.code) order by c.name --- !query 136 schema +-- !query 132 schema struct --- !query 136 output +-- !query 132 output A p 2 -1 B q 0 -1 C NULL NULL NULL --- !query 137 +-- !query 133 SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2861,13 +2833,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = sub2.key3 --- !query 137 schema +-- !query 133 schema struct --- !query 137 output +-- !query 133 output 1 1 1 1 --- !query 138 +-- !query 134 SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2883,13 +2855,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = sub2.key3 --- !query 138 schema +-- !query 134 schema struct --- !query 138 output +-- !query 134 output 1 1 1 1 --- !query 139 +-- !query 135 SELECT qq, unique1 FROM ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 @@ -2897,45 +2869,45 @@ SELECT qq, unique1 ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 USING (qq) INNER JOIN tenk1 c ON qq = unique2 --- !query 139 schema +-- !query 135 schema struct --- !query 139 output +-- !query 135 output 123 4596 123 4596 456 7318 --- !query 140 +-- !query 136 create or replace temporary view nt1 as select * from (values(1,true,true), (2,true,false), (3,false,false)) as v(id, a1, a2) --- !query 140 schema +-- !query 136 schema struct<> --- !query 140 output +-- !query 136 output --- !query 141 +-- !query 137 create or replace temporary view nt2 as select * from (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) as v(id, nt1_id, b1, b2) --- !query 141 schema +-- !query 137 schema struct<> --- !query 141 output +-- !query 137 output --- !query 142 +-- !query 138 create or replace temporary view nt3 as select * from (values(1,1,true), (2,2,false), (3,3,true)) as v(id, nt2_id, c1) --- !query 142 schema +-- !query 138 schema struct<> --- !query 142 output +-- !query 138 output --- !query 143 +-- !query 139 select nt3.id from nt3 as nt3 left join @@ -2947,17 +2919,17 @@ from nt3 as nt3 ) as ss2 on ss2.id = nt3.nt2_id where nt3.id = 1 and ss2.b3 --- !query 143 schema +-- !query 139 schema struct --- !query 143 output +-- !query 139 output 1 --- !query 144 +-- !query 140 select * from int4_tbl a full join int4_tbl b on true --- !query 144 schema +-- !query 140 schema struct --- !query 144 output +-- !query 140 output -123456 -123456 -123456 -2147483647 -123456 0 @@ -2985,11 +2957,11 @@ struct 2147483647 2147483647 --- !query 145 +-- !query 141 select * from int4_tbl a full join int4_tbl b on false --- !query 145 schema +-- !query 141 schema struct --- !query 145 output +-- !query 141 output -123456 NULL -2147483647 NULL 0 NULL @@ -3002,27 +2974,27 @@ NULL 123456 NULL 2147483647 --- !query 146 +-- !query 142 select count(*) from tenk1 a join tenk1 b on a.unique1 = b.unique2 left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand join int4_tbl on b.thousand = f1 --- !query 146 schema +-- !query 142 schema struct --- !query 146 output +-- !query 142 output 10 --- !query 147 +-- !query 143 select b.unique1 from tenk1 a join tenk1 b on a.unique1 = b.unique2 left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand join int4_tbl i1 on b.thousand = f1 right join int4_tbl i2 on i2.f1 = b.tenthous order by 1 --- !query 147 schema +-- !query 143 schema struct --- !query 147 output +-- !query 143 output NULL NULL NULL @@ -3030,7 +3002,7 @@ NULL 0 --- !query 148 +-- !query 144 select * from ( select unique1, q1, coalesce(unique1, -1) + q1 as fault @@ -3038,43 +3010,43 @@ select * from ) ss where fault = 122 order by fault --- !query 148 schema +-- !query 144 schema struct --- !query 148 output +-- !query 144 output NULL 123 122 --- !query 149 +-- !query 145 select q1, unique2, thousand, hundred from int8_tbl a left join tenk1 b on q1 = unique2 where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123) --- !query 149 schema +-- !query 145 schema struct --- !query 149 output +-- !query 145 output --- !query 150 +-- !query 146 select f1, unique2, case when unique2 is null then f1 else 0 end from int4_tbl a left join tenk1 b on f1 = unique2 where (case when unique2 is null then f1 else 0 end) = 0 --- !query 150 schema +-- !query 146 schema struct --- !query 150 output +-- !query 146 output 0 0 0 --- !query 151 +-- !query 147 select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44 --- !query 151 schema +-- !query 147 schema struct --- !query 151 output +-- !query 147 output --- !query 152 +-- !query 148 select * from text_tbl t1 inner join int8_tbl i8 @@ -3083,32 +3055,32 @@ select * from on t1.f1 = 'doh!' left join int4_tbl i4 on i8.q1 = i4.f1 --- !query 152 schema +-- !query 148 schema struct --- !query 152 output +-- !query 148 output doh! 123 456 doh! NULL doh! 123 456 hi de ho neighbor NULL --- !query 153 +-- !query 149 select * from (select 1 as id) as xx left join (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) on (xx.id = coalesce(yy.id)) --- !query 153 schema +-- !query 149 schema struct --- !query 153 output +-- !query 149 output 1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 --- !query 154 +-- !query 150 select a.q2, b.q1 from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) where coalesce(b.q1, 1) > 0 --- !query 154 schema +-- !query 150 schema struct --- !query 154 output +-- !query 150 output -4567890123456789 NULL 123 123 123 123 @@ -3121,124 +3093,124 @@ struct 4567890123456789 4567890123456789 --- !query 155 +-- !query 151 create or replace temporary view parent as select * from (values (1, 10), (2, 20), (3, 30)) as v(k, pd) --- !query 155 schema +-- !query 151 schema struct<> --- !query 155 output +-- !query 151 output --- !query 156 +-- !query 152 create or replace temporary view child as select * from (values (1, 100), (4, 400)) as v(k, cd) --- !query 156 schema +-- !query 152 schema struct<> --- !query 156 output +-- !query 152 output --- !query 157 +-- !query 153 select p.* from parent p left join child c on (p.k = c.k) --- !query 157 schema +-- !query 153 schema struct --- !query 157 output +-- !query 153 output 1 10 2 20 3 30 --- !query 158 +-- !query 154 select p.*, linked from parent p left join (select c.*, true as linked from child c) as ss on (p.k = ss.k) --- !query 158 schema +-- !query 154 schema struct --- !query 158 output +-- !query 154 output 1 10 true 2 20 NULL 3 30 NULL --- !query 159 +-- !query 155 select p.* from parent p left join child c on (p.k = c.k) where p.k = 1 and p.k = 2 --- !query 159 schema +-- !query 155 schema struct --- !query 159 output +-- !query 155 output --- !query 160 +-- !query 156 select p.* from (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k where p.k = 1 and p.k = 2 --- !query 160 schema +-- !query 156 schema struct --- !query 160 output +-- !query 156 output --- !query 161 +-- !query 157 create or replace temporary view a as select * from (values (0), (1)) as v(id) --- !query 161 schema +-- !query 157 schema struct<> --- !query 161 output +-- !query 157 output --- !query 162 +-- !query 158 create or replace temporary view b as select * from (values (0, 0), (1, NULL)) as v(id, a_id) --- !query 162 schema +-- !query 158 schema struct<> --- !query 162 output +-- !query 158 output --- !query 163 +-- !query 159 SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) --- !query 163 schema +-- !query 159 schema struct --- !query 163 output +-- !query 159 output 1 NULL NULL --- !query 164 +-- !query 160 SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) --- !query 164 schema +-- !query 160 schema struct --- !query 164 output +-- !query 160 output 1 NULL --- !query 165 +-- !query 161 create or replace temporary view innertab as select * from (values (123L, 42L)) as v(id, dat1) --- !query 165 schema +-- !query 161 schema struct<> --- !query 165 output +-- !query 161 output --- !query 166 +-- !query 162 SELECT * FROM (SELECT 1 AS x) ss1 LEFT JOIN (SELECT q1, q2, COALESCE(dat1, q1) AS y FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 ON true --- !query 166 schema +-- !query 162 schema struct --- !query 166 output +-- !query 162 output 1 123 456 123 1 123 4567890123456789 123 1 4567890123456789 -4567890123456789 4567890123456789 @@ -3246,163 +3218,163 @@ struct 1 4567890123456789 4567890123456789 4567890123456789 --- !query 167 +-- !query 163 select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1 --- !query 167 schema +-- !query 163 schema struct<> --- !query 167 output +-- !query 163 output org.apache.spark.sql.AnalysisException Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 63 --- !query 168 +-- !query 164 select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 --- !query 168 schema +-- !query 164 schema struct<> --- !query 168 output +-- !query 164 output org.apache.spark.sql.AnalysisException cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 --- !query 169 +-- !query 165 select * from int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 --- !query 169 schema +-- !query 165 schema struct --- !query 169 output +-- !query 165 output --- !query 170 +-- !query 166 select t1.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 170 schema +-- !query 166 schema struct<> --- !query 170 output +-- !query 166 output org.apache.spark.sql.AnalysisException cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 171 +-- !query 167 select t2.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 171 schema +-- !query 167 schema struct<> --- !query 171 output +-- !query 167 output org.apache.spark.sql.AnalysisException cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 172 +-- !query 168 select uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 172 schema +-- !query 168 schema struct<> --- !query 172 output +-- !query 168 output org.apache.spark.sql.AnalysisException cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 173 +-- !query 169 select f1,g from int4_tbl a, (select f1 as g) ss --- !query 173 schema +-- !query 169 schema struct<> --- !query 173 output +-- !query 169 output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 37 --- !query 174 +-- !query 170 select f1,g from int4_tbl a, (select a.f1 as g) ss --- !query 174 schema +-- !query 170 schema struct<> --- !query 174 output +-- !query 170 output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 37 --- !query 175 +-- !query 171 select f1,g from int4_tbl a cross join (select f1 as g) ss --- !query 175 schema +-- !query 171 schema struct<> --- !query 175 output +-- !query 171 output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 47 --- !query 176 +-- !query 172 select f1,g from int4_tbl a cross join (select a.f1 as g) ss --- !query 176 schema +-- !query 172 schema struct<> --- !query 176 output +-- !query 172 output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 47 --- !query 177 +-- !query 173 CREATE TABLE j1 (id1 int, id2 int) USING parquet --- !query 177 schema +-- !query 173 schema struct<> --- !query 177 output +-- !query 173 output --- !query 178 +-- !query 174 CREATE TABLE j2 (id1 int, id2 int) USING parquet --- !query 178 schema +-- !query 174 schema struct<> --- !query 178 output +-- !query 174 output --- !query 179 +-- !query 175 INSERT INTO j1 values(1,1),(1,2) --- !query 179 schema +-- !query 175 schema struct<> --- !query 179 output +-- !query 175 output --- !query 180 +-- !query 176 INSERT INTO j2 values(1,1) --- !query 180 schema +-- !query 176 schema struct<> --- !query 180 output +-- !query 176 output --- !query 181 +-- !query 177 INSERT INTO j2 values(1,2) --- !query 181 schema +-- !query 177 schema struct<> --- !query 181 output +-- !query 177 output --- !query 182 +-- !query 178 select * from j1 inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1 --- !query 182 schema +-- !query 178 schema struct --- !query 182 output +-- !query 178 output 1 1 1 1 1 2 1 2 --- !query 183 +-- !query 179 drop table j1 --- !query 183 schema +-- !query 179 schema struct<> --- !query 183 output +-- !query 179 output --- !query 184 +-- !query 180 drop table j2 --- !query 184 schema +-- !query 180 schema struct<> --- !query 184 output +-- !query 180 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/postgreSQL/numeric.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out 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/postgreSQL/select_distinct.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/select_distinct.sql.out 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/postgreSQL/select_having.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out 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/postgreSQL/select_implicit.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/strings.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out similarity index 86% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index db17ee46c6829..80322fb562895 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 30 +-- Number of queries: 32 -- !query 0 @@ -214,22 +214,50 @@ struct<49:string,d1:timestamp> -- !query 24 -SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' -- !query 24 schema -struct +struct<54:string,diff:interval> -- !query 24 output - 2004-02-23 00:00:00 + interval -1409 weeks -8 hours + interval 0 microseconds + interval 246 weeks 2 days 17 hours 19 minutes 20 seconds + interval 3 hours 4 minutes 5 seconds + interval 5 weeks 4 days 17 hours 32 minutes 1 seconds -- !query 25 +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +-- !query 25 schema +struct +-- !query 25 output + 2004-02-23 00:00:00 + + +-- !query 26 +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' +-- !query 26 schema +struct<54:string,diff:interval> +-- !query 26 output + interval -1409 weeks -8 hours + interval 0 microseconds + interval 246 weeks 2 days 17 hours 19 minutes 20 seconds + interval 3 hours 4 minutes 5 seconds + interval 5 weeks 4 days 17 hours 32 minutes 1 seconds + + +-- !query 27 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' --- !query 25 schema -struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:int> --- !query 25 output +-- !query 27 schema +struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:decimal(8,6)> +-- !query 27 output 1969-12-31 16:00:00 1969 12 31 16 0 0 1997-01-02 00:00:00 1997 1 2 0 0 0 1997-01-02 03:04:05 1997 1 2 3 4 5 @@ -237,14 +265,14 @@ struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute: 2001-09-22 18:19:20 2001 9 22 18 19 20 --- !query 26 +-- !query 28 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' --- !query 26 schema +-- !query 28 schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> --- !query 26 output +-- !query 28 output 1969-12-31 16:00:00 4 0 0 1997-01-02 00:00:00 1 0 0 1997-01-02 03:04:05 1 5000 5000000 @@ -252,14 +280,14 @@ struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> 2001-09-22 18:19:20 3 20000 20000000 --- !query 27 +-- !query 29 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' --- !query 27 schema +-- !query 29 schema struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> --- !query 27 output +-- !query 29 output 1969-12-31 16:00:00 1970 1 3 1997-01-02 00:00:00 1997 1 4 1997-01-02 03:04:05 1997 1 4 @@ -267,17 +295,17 @@ struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> 2001-09-22 18:19:20 2001 38 6 --- !query 28 +-- !query 30 SELECT make_timestamp(2014,12,28,6,30,45.887) --- !query 28 schema +-- !query 30 schema struct --- !query 28 output +-- !query 30 output 2014-12-28 06:30:45.887 --- !query 29 +-- !query 31 DROP TABLE TIMESTAMP_TBL --- !query 29 schema +-- !query 31 schema struct<> --- !query 29 output +-- !query 31 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/union.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out new file mode 100644 index 0000000000000..45bc98ae97640 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -0,0 +1,725 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 43 + + +-- !query 0 +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten +-- !query 1 schema +struct +-- !query 1 output +0 0 0 0.0 +0 2 0 2.0 +0 4 0 4.0 +0 6 0 6.0 +0 8 0 8.0 +1 1 2500 1.0 +1 3 2500 3.0 +1 5 2500 5.0 +1 7 2500 7.0 +1 9 2500 9.0 +2 0 5000 0.0 +2 2 5000 2.0 +2 4 5000 4.0 +2 6 5000 6.0 +2 8 5000 8.0 +3 1 7500 1.0 +3 3 7500 3.0 +3 5 7500 5.0 +3 7 7500 7.0 +3 9 7500 9.0 + + +-- !query 2 +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10 +-- !query 2 schema +struct +-- !query 2 output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query 3 +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS () +-- !query 3 schema +struct +-- !query 3 output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query 4 +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten) +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 5 schema +struct +-- !query 5 output +0 0 0 +0 0 0 +0 4 0 +1 7 1 +1 9 1 +2 0 2 +3 1 3 +3 3 3 +4 1 1 +5 1 1 + + +-- !query 6 +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 +-- !query 6 schema +struct +-- !query 6 output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 7 +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 7 schema +struct +-- !query 7 output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +3 4 0 +3 7 1 +4 9 1 + + +-- !query 8 +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 8 schema +struct +-- !query 8 output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +2 4 0 +2 7 1 +3 9 1 + + +-- !query 9 +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 9 schema +struct +-- !query 9 output +0.0 0 0 +0.0 0 0 +0.0 0 2 +0.0 1 1 +0.0 1 1 +0.0 1 3 +0.6666666666666666 7 1 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query 10 +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 10 schema +struct +-- !query 10 output +0.5 1 1 +0.5 1 1 +0.5 1 3 +0.6666666666666666 0 0 +0.6666666666666666 0 0 +0.75 7 1 +1.0 0 2 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query 11 +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 11 schema +struct +-- !query 11 output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +2 1 1 +2 1 3 +2 3 3 +3 4 0 +3 7 1 +3 9 1 + + +-- !query 12 +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 12 schema +struct +-- !query 12 output +0 0 0 +0 4 0 +1 1 1 +1 3 3 +1 7 1 +7 9 1 +NULL 0 0 +NULL 0 2 +NULL 1 1 +NULL 1 3 + + +-- !query 13 +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 13 schema +struct +-- !query 13 output +0 0 0 +1 1 1 +3 1 3 +4 0 0 +7 1 1 +9 7 1 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query 14 +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 14 schema +struct +-- !query 14 output +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query 15 +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 15 schema +struct +-- !query 15 output +-1 0 2 +-1 3 3 +-1 4 0 +-1 9 1 +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 + + +-- !query 16 +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 16 schema +struct +-- !query 16 output +0 0 0 +0 0 0 +0 0 2 +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 3 3 +1 7 1 +1 9 1 + + +-- !query 17 +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 17 schema +struct +-- !query 17 output +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 7 1 +1 9 1 +2 0 0 +2 0 0 +2 0 2 +3 3 3 + + +-- !query 18 +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten +-- !query 18 schema +struct +-- !query 18 output +4 0 0 +4 0 0 +4 4 0 +9 1 1 +9 1 1 +9 7 1 +9 9 1 +0 0 2 +3 1 3 +3 3 3 + + +-- !query 19 +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two +-- !query 19 schema +struct +-- !query 19 output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query 20 +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 +-- !query 20 schema +struct +-- !query 20 output +2 3 +2 3 +4 1 +4 1 +4 1 +4 1 + + +-- !query 21 +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10 +-- !query 21 schema +struct +-- !query 21 output +136 +22 +22 +24 +24 +51 +82 +87 +92 +92 + + +-- !query 22 +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum +-- !query 22 schema +struct +-- !query 22 output + + + +-- !query 23 +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 +-- !query 23 schema +struct +-- !query 23 output +0.0 +0.0 +0.0 +1.0 +1.0 +1.0 +1.0 +2.0 +3.0 +3.0 + + +-- !query 24 +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten) +-- !query 24 schema +struct +-- !query 24 output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query 25 +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s +-- !query 25 schema +struct +-- !query 25 output + + + +-- !query 26 +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 +-- !query 27 schema +struct +-- !query 27 output +0 + + +-- !query 28 +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten +-- !query 28 schema +struct +-- !query 28 output +0 9976146 4 +1 10114187 9 +2 10059554 8 +3 9878541 1 +4 9881005 2 +5 9981670 5 +6 9947099 3 +7 10120309 10 +8 9991305 6 +9 10040184 7 + + +-- !query 29 +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss +-- !query 29 schema +struct +-- !query 29 output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query 30 +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss +-- !query 30 schema +struct +-- !query 30 output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query 31 +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss +-- !query 31 schema +struct +-- !query 31 output +0 0 20 8 +0 2 20 8 +0 4 20 8 +0 6 20 8 +0 8 20 8 +1 1 25 9 +1 3 25 9 +1 5 25 9 +1 7 25 9 +1 9 25 9 +2 0 20 8 +2 2 20 8 +2 4 20 8 +2 6 20 8 +2 8 20 8 +3 1 25 9 +3 3 25 9 +3 5 25 9 +3 7 25 9 +3 9 25 9 + + +-- !query 32 +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 32 schema +struct +-- !query 32 output +10 3 3 +10 7 3 +18 2 2 +18 6 2 +33 1 1 +33 5 1 +33 9 1 +45 0 0 +45 4 0 +45 8 0 + + +-- !query 33 +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 33 schema +struct +-- !query 33 output +0 0 0 +10 3 3 +15 5 1 +23 8 0 +32 9 1 +38 6 2 +39 1 1 +41 2 2 +45 4 0 +7 7 3 + + +-- !query 34 +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 34 schema +struct +-- !query 34 output +10 0 0 +13 2 2 +15 7 3 +22 1 1 +23 3 3 +26 6 2 +29 9 1 +31 8 0 +32 5 1 +7 4 0 + + +-- !query 35 +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 35 schema +struct +-- !query 35 output +10 0 0 +13 3 3 +15 8 0 +17 5 1 +3 6 2 +4 2 2 +6 1 1 +7 9 1 +8 7 3 +NULL 4 0 + + +-- !query 36 +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 36 schema +struct +-- !query 36 output +0 7 3 +10 5 1 +15 8 0 +16 2 2 +16 9 1 +22 6 2 +23 1 1 +7 3 3 +9 4 0 +NULL 0 0 + + +-- !query 37 +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 37 schema +struct +-- !query 37 output +13 1 1 +22 6 2 +30 9 1 +35 8 0 +38 5 1 +45 0 0 +45 3 3 +45 7 3 +6 4 0 +7 2 2 + + +-- !query 38 +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i +-- !query 38 schema +struct<> +-- !query 38 output + + + +-- !query 39 +SELECT * FROM v_window +-- !query 39 schema +struct +-- !query 39 output +1 3 +10 19 +2 6 +3 9 +4 12 +5 15 +6 18 +7 21 +8 24 +9 27 + + +-- !query 40 +DROP VIEW v_window +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +DROP VIEW tenk2 +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +DROP VIEW int4_tbl +-- !query 42 schema +struct<> +-- !query 42 output + diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out new file mode 100644 index 0000000000000..0d2c78847b97c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -0,0 +1,479 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 31 + + +-- !query 0 +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 1 schema +struct +-- !query 1 output +12 1 1 +12 5 1 +12 9 1 +23 3 3 +23 7 3 +27 2 2 +27 6 2 +NULL 0 0 +NULL 4 0 +NULL 8 0 + + +-- !query 2 +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 2 schema +struct +-- !query 2 output +10 2 2 +10 6 2 +18 1 1 +18 5 1 +18 9 1 +23 0 0 +23 4 0 +23 8 0 +NULL 3 3 +NULL 7 3 + + +-- !query 3 +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 3 schema +struct +-- !query 3 output +10 3 3 +10 7 3 +12 4 0 +12 8 0 +14 9 1 +15 5 1 +4 0 0 +6 1 1 +8 2 2 +8 6 2 + + +-- !query 4 +select last(salary) over(order by salary range between 1000 preceding and 1000 following), +lag(salary) over(order by salary range between 1000 preceding and 1000 following), +salary from empsalary +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1); + + +-- !query 5 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following) +-- !query 5 schema +struct +-- !query 5 output +1 1 1 3 +2 2 1 4 +3 3 1 5 +4 4 2 5 +5 5 3 5 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 6 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following) +-- !query 6 schema +struct +-- !query 6 output +1 1 1 3 +2 2 1 4 +3 3 1 5 +4 4 2 5 +5 5 3 5 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 7 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following) +-- !query 7 schema +struct +-- !query 7 output +1 1 3 1 +2 2 4 1 +3 3 5 1 +4 4 5 2 +5 5 5 3 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 8 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following) +-- !query 8 schema +struct +-- !query 8 output +1 1 3 1 +2 2 4 1 +3 3 5 1 +4 4 5 2 +5 5 5 3 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 9 +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x +-- !query 9 schema +struct +-- !query 9 output +32764 32766 +32765 32766 +32766 32766 + + +-- !query 10 +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x +-- !query 10 schema +struct +-- !query 10 output +-32766 -32766 + + +-- !query 11 +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x +-- !query 11 schema +struct +-- !query 11 output +2147483644 2147483646 +2147483645 2147483646 +2147483646 2147483646 + + +-- !query 12 +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x +-- !query 12 schema +struct +-- !query 12 output +-2147483646 -2147483646 + + +-- !query 13 +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x +-- !query 13 schema +struct<> +-- !query 13 output +java.lang.ArithmeticException +long overflow + + +-- !query 14 +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x +-- !query 14 schema +struct<> +-- !query 14 output +java.lang.ArithmeticException +long overflow + + +-- !query 15 +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100) +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following) +-- !query 17 schema +struct +-- !query 17 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 3 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 18 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following) +-- !query 18 schema +struct +-- !query 18 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 4 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 19 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following) +-- !query 19 schema +struct +-- !query 19 output +1 -3.0 1 7 +2 -1.0 1 7 +3 0.0 1 7 +4 1.1 1 7 +5 1.12 1 7 +6 2.0 1 7 +7 100.0 1 7 + + +-- !query 20 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following) +-- !query 20 schema +struct +-- !query 20 output +1 -3.0 1 7 +2 -1.0 2 7 +3 0.0 2 7 +4 1.1 3 7 +5 1.12 4 7 +6 2.0 4 7 +7 100.0 7 7 + + +-- !query 21 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following) +-- !query 21 schema +struct +-- !query 21 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 3 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 22 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following) +-- !query 22 schema +struct +-- !query 22 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 4 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 23 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following) +-- !query 23 schema +struct +-- !query 23 output +1 -3.0 1 7 +2 -1.0 1 7 +3 0.0 1 7 +4 1.1 1 7 +5 1.12 1 7 +6 2.0 1 7 +7 100.0 1 7 + + +-- !query 24 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following) +-- !query 24 schema +struct +-- !query 24 output +1 -3.0 1 7 +2 -1.0 2 7 +3 0.0 2 7 +4 1.1 3 7 +5 1.12 4 7 +6 2.0 4 7 +7 100.0 7 7 + + +-- !query 25 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following) +-- !query 25 schema +struct +-- !query 25 output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query 26 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query 26 schema +struct +-- !query 26 output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query 27 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query 27 schema +struct +-- !query 27 output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query 28 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following) +-- !query 28 schema +struct +-- !query 28 output +1 -3 NULL NULL +2 -1 NULL NULL +3 0 NULL NULL +4 1 NULL NULL +5 1 NULL NULL +6 2 NULL NULL +7 100 NULL NULL + + +-- !query 29 +drop table empsalary +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +drop table numerics +-- !query 30 schema +struct<> +-- !query 30 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/postgreSQL/with.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out rename to sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out 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 f22cb7e200e6c..ee05e2413799a 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 @@ -140,7 +140,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type: MANAGED Provider: parquet -Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1 +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1 Partition Provider: Catalog Partition Columns: [`c`, `d`] Schema: root @@ -157,7 +157,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type: MANAGED Provider: parquet -Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t2 +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t2 Schema: root |-- b: string (nullable = true) |-- d: integer (nullable = true) @@ -183,7 +183,7 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1) struct -- !query 14 output showdb show_t1 false Partition Values: [c=Us, d=1] -Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1 +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1/c=Us/d=1 Created Time [not included in comparison] Last Access [not included in comparison] diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out index ab6a11a2b7efa..7089fbd25e28c 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -102,7 +102,7 @@ GROUP BY t1a, t3a, t3b, t3c -ORDER BY t1a DESC, t3b DESC +ORDER BY t1a DESC, t3b DESC, t3c ASC -- !query 4 schema struct -- !query 4 output diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out index bae5d00cc8632..498378bfa5c37 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out @@ -169,7 +169,7 @@ GROUP BY t1b, HAVING t1d NOT IN (SELECT t2d FROM t2 WHERE t1d = t2d) -ORDER BY t1b DESC +ORDER BY t1b DESC, t1d ASC -- !query 6 schema struct -- !query 6 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/postgreSQL/udf-aggregates_part1.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out 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/postgreSQL/udf-aggregates_part2.sql.out similarity index 57% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out index ad2f1bdf77d7a..3b54637c449d8 100644 --- 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/postgreSQL/udf-aggregates_part2.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 16 +-- Number of queries: 25 -- !query 0 @@ -17,6 +17,50 @@ struct<> -- !query 1 +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query 2 schema +struct +-- !query 2 output +NULL NULL + + +-- !query 3 +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query 3 schema +struct +-- !query 3 output +NULL NULL + + +-- !query 4 +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query 4 schema +struct +-- !query 4 output +1 1 1 1 7 7 7 3 + + +-- !query 5 SELECT (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, @@ -27,13 +71,13 @@ SELECT NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, NOT (FALSE AND FALSE) AS `t` --- !query 1 schema +-- !query 5 schema struct --- !query 1 output +-- !query 5 output true true false true false true true true true --- !query 2 +-- !query 6 SELECT (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, @@ -44,113 +88,177 @@ SELECT (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, NOT (FALSE OR FALSE) AS `t` --- !query 2 schema +-- !query 6 schema struct --- !query 2 output +-- !query 6 output true false true false true true true true true --- !query 3 +-- !query 7 +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 +-- !query 8 schema +struct +-- !query 8 output +NULL NULL + + +-- !query 9 +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test +-- !query 9 schema +struct +-- !query 9 output +false true false NULL false true + + +-- !query 10 +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test +-- !query 10 schema +struct +-- !query 10 output +false true false NULL false true + + +-- !query 11 +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test +-- !query 11 schema +struct +-- !query 11 output +true true false NULL false true + + +-- !query 12 select min(udf(unique1)) from tenk1 --- !query 3 schema +-- !query 12 schema struct --- !query 3 output +-- !query 12 output 0 --- !query 4 +-- !query 13 select udf(max(unique1)) from tenk1 --- !query 4 schema +-- !query 13 schema struct --- !query 4 output +-- !query 13 output 9999 --- !query 5 +-- !query 14 select max(unique1) from tenk1 where udf(unique1) < 42 --- !query 5 schema +-- !query 14 schema struct --- !query 5 output +-- !query 14 output 41 --- !query 6 +-- !query 15 select max(unique1) from tenk1 where unique1 > udf(42) --- !query 6 schema +-- !query 15 schema struct --- !query 6 output +-- !query 15 output 9999 --- !query 7 +-- !query 16 select max(unique1) from tenk1 where udf(unique1) > 42000 --- !query 7 schema +-- !query 16 schema struct --- !query 7 output +-- !query 16 output NULL --- !query 8 +-- !query 17 select max(tenthous) from tenk1 where udf(thousand) = 33 --- !query 8 schema +-- !query 17 schema struct --- !query 8 output +-- !query 17 output 9033 --- !query 9 +-- !query 18 select min(tenthous) from tenk1 where udf(thousand) = 33 --- !query 9 schema +-- !query 18 schema struct --- !query 9 output +-- !query 18 output 33 --- !query 10 +-- !query 19 select distinct max(udf(unique2)) from tenk1 --- !query 10 schema +-- !query 19 schema struct --- !query 10 output +-- !query 19 output 9999 --- !query 11 +-- !query 20 select max(unique2) from tenk1 order by udf(1) --- !query 11 schema +-- !query 20 schema struct --- !query 11 output +-- !query 20 output 9999 --- !query 12 +-- !query 21 select max(unique2) from tenk1 order by max(udf(unique2)) --- !query 12 schema +-- !query 21 schema struct --- !query 12 output +-- !query 21 output 9999 --- !query 13 +-- !query 22 select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 --- !query 13 schema +-- !query 22 schema struct --- !query 13 output +-- !query 22 output 9999 --- !query 14 +-- !query 23 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 +-- !query 23 schema struct --- !query 14 output +-- !query 23 output 9999 3 9999 2 9999 1 --- !query 15 +-- !query 24 select udf(max(100)) from tenk1 --- !query 15 schema +-- !query 24 schema struct --- !query 15 output +-- !query 24 output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part3.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part4.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out 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/postgreSQL/udf-case.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out rename to sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index febe47b5ba84e..b762d18fb8390 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL -- !query 32 SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL -- !query 33 SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true @@ -317,7 +317,7 @@ false true true -- !query 34 SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) -- !query 34 schema -struct +struct -- !query 34 output 1 false true true 2 true true true @@ -389,7 +389,7 @@ SELECT some(udf(1S)) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'any(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -430,7 +430,7 @@ struct +struct -- !query 44 output 1 false false 1 true true diff --git a/sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv b/sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv new file mode 100644 index 0000000000000..cabb50e9608e6 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv @@ -0,0 +1,4 @@ +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/cars-multichar-delim.csv b/sql/core/src/test/resources/test-data/cars-multichar-delim.csv new file mode 100644 index 0000000000000..4309edbf04418 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-multichar-delim.csv @@ -0,0 +1,4 @@ +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/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index a4b142b7ab78e..29766c4651d7d 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 @@ -124,20 +124,24 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession test("percentile_approx, with different accuracies") { withTempView(table) { - (1 to 1000).toDF("col").createOrReplaceTempView(table) + val tableCount = 1000 + (1 to tableCount).toDF("col").createOrReplaceTempView(table) // With different accuracies - val expectedPercentile = 250D val accuracies = Array(1, 10, 100, 1000, 10000) - val errors = accuracies.map { accuracy => - val df = spark.sql(s"SELECT percentile_approx(col, 0.25, $accuracy) FROM $table") - val approximatePercentile = df.collect().head.getInt(0) - val error = Math.abs(approximatePercentile - expectedPercentile) - error + val expectedPercentiles = Array(100D, 200D, 250D, 314D, 777D) + for (accuracy <- accuracies) { + for (expectedPercentile <- expectedPercentiles) { + val df = spark.sql( + s"""SELECT + | percentile_approx(col, $expectedPercentile/$tableCount, $accuracy) + |FROM $table + """.stripMargin) + val approximatePercentile = df.collect().head.getInt(0) + val error = Math.abs(approximatePercentile - expectedPercentile) + assert(error <= math.floor(tableCount.toDouble / accuracy.toDouble)) + } } - - // The larger accuracy value we use, the smaller error we get - assert(errors.sorted.sameElements(errors.reverse)) } } 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 9bb741bee4e40..07afd4195c3d4 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 @@ -43,7 +43,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { } } - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() RuleExecutor.resetMetrics() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala index 4d0eb04be751b..6b503334f9f23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala @@ -23,14 +23,14 @@ import org.apache.spark.sql.test.SharedSparkSession class ComplexTypesSuite extends QueryTest with SharedSparkSession { - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() spark.range(10).selectExpr( "id + 1 as i1", "id + 2 as i2", "id + 3 as i3", "id + 4 as i4", "id + 5 as i5") .write.saveAsTable("tab") } - override def afterAll() { + override def afterAll(): Unit = { try { spark.sql("DROP TABLE IF EXISTS tab") } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index e9179a39d3b6d..4f25642906628 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -18,8 +18,12 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.DefinedByConstructorParams +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.objects.MapObjects import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.ArrayType /** * A test suite to test DataFrame/SQL functionalities with complex types (i.e. array, struct, map). @@ -64,6 +68,24 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession { val ds100_5 = Seq(S100_5()).toDS() ds100_5.rdd.count } + + test("SPARK-29503 nest unsafe struct inside safe array") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = spark.sparkContext.parallelize(Seq(Seq(1, 2, 3))).toDF("items") + + // items: Seq[Int] => items.map { item => Seq(Struct(item)) } + val result = df.select( + new Column(MapObjects( + (item: Expression) => array(struct(new Column(item))).expr, + $"items".expr, + df.schema("items").dataType.asInstanceOf[ArrayType].elementType + )) as "items" + ).collect() + + assert(result.size === 1) + assert(result === Row(Seq(Seq(Row(1)), Seq(Row(2)), Seq(Row(3)))) :: Nil) + } + } } class S100( 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 7d044638db571..06484908f5e73 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 @@ -1930,6 +1930,18 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq(5, 9, 11, 10, 6)), Row(Seq.empty), Row(null))) + checkAnswer(df.select(transform(col("i"), x => x + 1)), + Seq( + Row(Seq(2, 10, 9, 8)), + Row(Seq(6, 9, 10, 8, 3)), + Row(Seq.empty), + Row(null))) + checkAnswer(df.select(transform(col("i"), (x, i) => x + i)), + Seq( + Row(Seq(1, 10, 10, 10)), + Row(Seq(5, 9, 11, 10, 6)), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -1960,6 +1972,18 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq(5, null, 10, 12, 11, 7)), Row(Seq.empty), Row(null))) + checkAnswer(df.select(transform(col("i"), x => x + 1)), + Seq( + Row(Seq(2, 10, 9, null, 8)), + Row(Seq(6, null, 9, 10, 8, 3)), + Row(Seq.empty), + Row(null))) + checkAnswer(df.select(transform(col("i"), (x, i) => x + i)), + Seq( + Row(Seq(1, 10, 10, null, 11)), + Row(Seq(5, null, 10, 12, 11, 7)), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -1990,6 +2014,18 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq("b0", null, "c2", null)), Row(Seq.empty), Row(null))) + checkAnswer(df.select(transform(col("s"), x => concat(x, x))), + Seq( + Row(Seq("cc", "aa", "bb")), + Row(Seq("bb", null, "cc", null)), + Row(Seq.empty), + Row(null))) + checkAnswer(df.select(transform(col("s"), (x, i) => concat(x, i))), + Seq( + Row(Seq("c0", "a1", "b2")), + Row(Seq("b0", null, "c2", null)), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2034,6 +2070,32 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Seq("b", null, "c", null, null))), Row(Seq.empty), Row(null))) + checkAnswer(df.select(transform(col("arg"), arg => arg)), + Seq( + Row(Seq("c", "a", "b")), + Row(Seq("b", null, "c", null)), + Row(Seq.empty), + Row(null))) + checkAnswer(df.select(transform(col("arg"), _ => col("arg"))), + Seq( + Row(Seq(Seq("c", "a", "b"), Seq("c", "a", "b"), Seq("c", "a", "b"))), + Row(Seq( + Seq("b", null, "c", null), + Seq("b", null, "c", null), + Seq("b", null, "c", null), + Seq("b", null, "c", null))), + Row(Seq.empty), + Row(null))) + checkAnswer(df.select(transform(col("arg"), x => concat(col("arg"), array(x)))), + Seq( + Row(Seq(Seq("c", "a", "b", "c"), Seq("c", "a", "b", "a"), Seq("c", "a", "b", "b"))), + Row(Seq( + Seq("b", null, "c", null, "b"), + Seq("b", null, "c", null, null), + Seq("b", null, "c", null, "c"), + Seq("b", null, "c", null, null))), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2080,6 +2142,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Map(), Map(1 -> -1, 2 -> -2, 3 -> -3)), Row(Map(1 -> 10), Map(3 -> -3)))) + checkAnswer(dfInts.select( + map_filter(col("m"), (k, v) => k * 10 === v), + map_filter(col("m"), (k, v) => k === (v * -1))), + Seq( + Row(Map(1 -> 10, 2 -> 20, 3 -> 30), Map()), + Row(Map(), Map(1 -> -1, 2 -> -2, 3 -> -3)), + Row(Map(1 -> 10), Map(3 -> -3)))) + val dfComplex = Seq( Map(1 -> Seq(Some(1)), 2 -> Seq(Some(1), Some(2)), 3 -> Seq(Some(1), Some(2), Some(3))), Map(1 -> null, 2 -> Seq(Some(-2), Some(-2)), 3 -> Seq[Option[Int]](None))).toDF("m") @@ -2090,6 +2160,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Map(1 -> Seq(1)), Map(1 -> Seq(1), 2 -> Seq(1, 2), 3 -> Seq(1, 2, 3))), Row(Map(), Map(2 -> Seq(-2, -2))))) + checkAnswer(dfComplex.select( + map_filter(col("m"), (k, v) => k === element_at(v, 1)), + map_filter(col("m"), (k, v) => k === size(v))), + Seq( + Row(Map(1 -> Seq(1)), Map(1 -> Seq(1), 2 -> Seq(1, 2), 3 -> Seq(1, 2, 3))), + Row(Map(), Map(2 -> Seq(-2, -2))))) + // Invalid use cases val df = Seq( (Map(1 -> "a"), 1), @@ -2112,6 +2189,11 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } assert(ex3.getMessage.contains("data type mismatch: argument 1 requires map type")) + val ex3a = intercept[AnalysisException] { + df.select(map_filter(col("i"), (k, v) => k > v)) + } + assert(ex3a.getMessage.contains("data type mismatch: argument 1 requires map type")) + val ex4 = intercept[AnalysisException] { df.selectExpr("map_filter(a, (k, v) -> k > v)") } @@ -2133,6 +2215,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq(8, 2)), Row(Seq.empty), Row(null))) + checkAnswer(df.select(filter(col("i"), _ % 2 === 0)), + Seq( + Row(Seq(8)), + Row(Seq(8, 2)), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2157,6 +2245,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq(8, 2)), Row(Seq.empty), Row(null))) + checkAnswer(df.select(filter(col("i"), _ % 2 === 0)), + Seq( + Row(Seq(8)), + Row(Seq(8, 2)), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2181,6 +2275,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq("b", "c")), Row(Seq.empty), Row(null))) + checkAnswer(df.select(filter(col("s"), x => x.isNotNull)), + Seq( + Row(Seq("c", "a", "b")), + Row(Seq("b", "c")), + Row(Seq.empty), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2190,6 +2290,36 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { testNonPrimitiveType() } + test("filter function - index argument") { + val df = Seq( + Seq("c", "a", "b"), + Seq("b", null, "c", null), + Seq.empty, + null + ).toDF("s") + + def testIndexArgument(): Unit = { + checkAnswer(df.selectExpr("filter(s, (x, i) -> i % 2 == 0)"), + Seq( + Row(Seq("c", "b")), + Row(Seq("b", "c")), + Row(Seq.empty), + Row(null))) + checkAnswer(df.select(filter(col("s"), (x, i) => i % 2 === 0)), + Seq( + Row(Seq("c", "b")), + Row(Seq("b", "c")), + Row(Seq.empty), + Row(null))) + } + + // Test with local relation, the Project will be evaluated without codegen + testIndexArgument() + // Test with cached relation, the Project will be evaluated with codegen + df.cache() + testIndexArgument() + } + test("filter function - invalid") { val df = Seq( (Seq("c", "a", "b"), 1), @@ -2199,20 +2329,30 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).toDF("s", "i") val ex1 = intercept[AnalysisException] { - df.selectExpr("filter(s, (x, y) -> x + y)") + df.selectExpr("filter(s, (x, y, z) -> x + y)") } - assert(ex1.getMessage.contains("The number of lambda function arguments '2' does not match")) + assert(ex1.getMessage.contains("The number of lambda function arguments '3' does not match")) val ex2 = intercept[AnalysisException] { df.selectExpr("filter(i, x -> x)") } assert(ex2.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex2a = intercept[AnalysisException] { + df.select(filter(col("i"), x => x)) + } + assert(ex2a.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex3 = intercept[AnalysisException] { df.selectExpr("filter(s, x -> x)") } assert(ex3.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + val ex3a = intercept[AnalysisException] { + df.select(filter(col("s"), x => x)) + } + assert(ex3a.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + val ex4 = intercept[AnalysisException] { df.selectExpr("filter(a, x -> x)") } @@ -2234,6 +2374,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(false), Row(false), Row(null))) + checkAnswer(df.select(exists(col("i"), _ % 2 === 0)), + Seq( + Row(true), + Row(false), + Row(false), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2260,6 +2406,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(null), Row(false), Row(null))) + checkAnswer(df.select(exists(col("i"), _ % 2 === 0)), + Seq( + Row(true), + Row(false), + Row(null), + Row(false), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2284,6 +2437,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(true), Row(false), Row(null))) + checkAnswer(df.select(exists(col("s"), x => x.isNull)), + Seq( + Row(false), + Row(true), + Row(false), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2311,11 +2470,21 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } assert(ex2.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex2a = intercept[AnalysisException] { + df.select(exists(col("i"), x => x)) + } + assert(ex2.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex3 = intercept[AnalysisException] { df.selectExpr("exists(s, x -> x)") } assert(ex3.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + val ex3a = intercept[AnalysisException] { + df.select(exists(df("s"), x => x)) + } + assert(ex3a.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + val ex4 = intercept[AnalysisException] { df.selectExpr("exists(a, x -> x)") } @@ -2337,6 +2506,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(true), Row(true), Row(null))) + checkAnswer(df.select(forall(col("i"), x => x % 2 === 0)), + Seq( + Row(false), + Row(true), + Row(true), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2363,6 +2538,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(true), Row(true), Row(null))) + checkAnswer(df.select(forall(col("i"), x => (x % 2 === 0) || x.isNull)), + Seq( + Row(false), + Row(true), + Row(true), + Row(true), + Row(null))) checkAnswer(df.selectExpr("forall(i, x -> x % 2 == 0)"), Seq( Row(false), @@ -2370,6 +2552,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(true), Row(true), Row(null))) + checkAnswer(df.select(forall(col("i"), x => x % 2 === 0)), + Seq( + Row(false), + Row(null), + Row(true), + Row(true), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2394,6 +2583,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(true), Row(true), Row(null))) + checkAnswer(df.select(forall(col("s"), _.isNull)), + Seq( + Row(false), + Row(true), + Row(true), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2421,15 +2616,30 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } assert(ex2.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex2a = intercept[AnalysisException] { + df.select(forall(col("i"), x => x)) + } + assert(ex2a.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex3 = intercept[AnalysisException] { df.selectExpr("forall(s, x -> x)") } assert(ex3.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + val ex3a = intercept[AnalysisException] { + df.select(forall(col("s"), x => x)) + } + assert(ex3a.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + val ex4 = intercept[AnalysisException] { df.selectExpr("forall(a, x -> x)") } assert(ex4.getMessage.contains("cannot resolve '`a`'")) + + val ex4a = intercept[AnalysisException] { + df.select(forall(col("a"), x => x)) + } + assert(ex4a.getMessage.contains("cannot resolve '`a`'")) } test("aggregate function - array for primitive type not containing null") { @@ -2453,6 +2663,18 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(310), Row(0), Row(null))) + checkAnswer(df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x)), + Seq( + Row(25), + Row(31), + Row(0), + Row(null))) + checkAnswer(df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x, _ * 10)), + Seq( + Row(250), + Row(310), + Row(0), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2484,6 +2706,20 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(0), Row(0), Row(null))) + checkAnswer(df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x)), + Seq( + Row(25), + Row(null), + Row(0), + Row(null))) + checkAnswer( + df.select( + aggregate(col("i"), lit(0), (acc, x) => acc + x, acc => coalesce(acc, lit(0)) * 10)), + Seq( + Row(250), + Row(0), + Row(0), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2515,6 +2751,21 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(""), Row("c"), Row(null))) + checkAnswer(df.select(aggregate(col("ss"), col("s"), (acc, x) => concat(acc, x))), + Seq( + Row("acab"), + Row(null), + Row("c"), + Row(null))) + checkAnswer( + df.select( + aggregate(col("ss"), col("s"), (acc, x) => concat(acc, x), + acc => coalesce(acc, lit("")))), + Seq( + Row("acab"), + Row(""), + Row("c"), + Row(null))) } // Test with local relation, the Project will be evaluated without codegen @@ -2547,11 +2798,21 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } assert(ex3.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex3a = intercept[AnalysisException] { + df.select(aggregate(col("i"), lit(0), (acc, x) => x)) + } + assert(ex3a.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex4 = intercept[AnalysisException] { df.selectExpr("aggregate(s, 0, (acc, x) -> x)") } assert(ex4.getMessage.contains("data type mismatch: argument 3 requires int type")) + val ex4a = intercept[AnalysisException] { + df.select(aggregate(col("s"), lit(0), (acc, x) => x)) + } + assert(ex4a.getMessage.contains("data type mismatch: argument 3 requires int type")) + val ex5 = intercept[AnalysisException] { df.selectExpr("aggregate(a, 0, (acc, x) -> x)") } @@ -2572,6 +2833,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Map(10 -> null, 8 -> false, 4 -> null)), Row(Map(5 -> null)), Row(null))) + + checkAnswer(df.select(map_zip_with(df("m1"), df("m2"), (k, v1, v2) => k === v1 + v2)), + Seq( + Row(Map(8 -> true, 3 -> false, 6 -> true)), + Row(Map(10 -> null, 8 -> false, 4 -> null)), + Row(Map(5 -> null)), + Row(null))) } test("map_zip_with function - map of non-primitive types") { @@ -2588,6 +2856,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Map("b" -> Row("a", null), "c" -> Row("d", "a"), "d" -> Row(null, "k"))), Row(Map("a" -> Row("d", null))), Row(null))) + + checkAnswer(df.select(map_zip_with(col("m1"), col("m2"), (k, v1, v2) => struct(v1, v2))), + Seq( + Row(Map("z" -> Row("a", "c"), "y" -> Row("b", null), "x" -> Row("c", "a"))), + Row(Map("b" -> Row("a", null), "c" -> Row("d", "a"), "d" -> Row(null, "k"))), + Row(Map("a" -> Row("d", null))), + Row(null))) } test("map_zip_with function - invalid") { @@ -2606,16 +2881,32 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { assert(ex2.getMessage.contains("The input to function map_zip_with should have " + "been two maps with compatible key types")) + val ex2a = intercept[AnalysisException] { + df.select(map_zip_with(df("mis"), col("mmi"), (x, y, z) => concat(x, y, z))) + } + assert(ex2a.getMessage.contains("The input to function map_zip_with should have " + + "been two maps with compatible key types")) + val ex3 = intercept[AnalysisException] { df.selectExpr("map_zip_with(i, mis, (x, y, z) -> concat(x, y, z))") } assert(ex3.getMessage.contains("type mismatch: argument 1 requires map type")) + val ex3a = intercept[AnalysisException] { + df.select(map_zip_with(col("i"), col("mis"), (x, y, z) => concat(x, y, z))) + } + assert(ex3a.getMessage.contains("type mismatch: argument 1 requires map type")) + val ex4 = intercept[AnalysisException] { df.selectExpr("map_zip_with(mis, i, (x, y, z) -> concat(x, y, z))") } assert(ex4.getMessage.contains("type mismatch: argument 2 requires map type")) + val ex4a = intercept[AnalysisException] { + df.select(map_zip_with(col("mis"), col("i"), (x, y, z) => concat(x, y, z))) + } + assert(ex4a.getMessage.contains("type mismatch: argument 2 requires map type")) + val ex5 = intercept[AnalysisException] { df.selectExpr("map_zip_with(mmi, mmi, (x, y, z) -> x)") } @@ -2644,27 +2935,59 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(dfExample1.selectExpr("transform_keys(i, (k, v) -> k + v)"), Seq(Row(Map(2 -> 1, 18 -> 9, 16 -> 8, 14 -> 7)))) + checkAnswer(dfExample1.select(transform_keys(col("i"), (k, v) => k + v)), + Seq(Row(Map(2 -> 1, 18 -> 9, 16 -> 8, 14 -> 7)))) + checkAnswer(dfExample2.selectExpr("transform_keys(j, " + "(k, v) -> map_from_arrays(ARRAY(1, 2, 3), ARRAY('one', 'two', 'three'))[k])"), Seq(Row(Map("one" -> 1.0, "two" -> 1.4, "three" -> 1.7)))) + checkAnswer(dfExample2.select( + transform_keys( + col("j"), + (k, v) => element_at( + map_from_arrays( + array(lit(1), lit(2), lit(3)), + array(lit("one"), lit("two"), lit("three")) + ), + k + ) + ) + ), + Seq(Row(Map("one" -> 1.0, "two" -> 1.4, "three" -> 1.7)))) + checkAnswer(dfExample2.selectExpr("transform_keys(j, (k, v) -> CAST(v * 2 AS BIGINT) + k)"), Seq(Row(Map(3 -> 1.0, 4 -> 1.4, 6 -> 1.7)))) + checkAnswer(dfExample2.select(transform_keys(col("j"), + (k, v) => (v * 2).cast("bigint") + k)), + Seq(Row(Map(3 -> 1.0, 4 -> 1.4, 6 -> 1.7)))) + checkAnswer(dfExample2.selectExpr("transform_keys(j, (k, v) -> k + v)"), Seq(Row(Map(2.0 -> 1.0, 3.4 -> 1.4, 4.7 -> 1.7)))) + checkAnswer(dfExample2.select(transform_keys(col("j"), (k, v) => k + v)), + Seq(Row(Map(2.0 -> 1.0, 3.4 -> 1.4, 4.7 -> 1.7)))) + checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> k % 2 = 0 OR v)"), Seq(Row(Map(true -> true, true -> false)))) + checkAnswer(dfExample3.select(transform_keys(col("x"), (k, v) => k % 2 === 0 || v)), + Seq(Row(Map(true -> true, true -> false)))) + checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * k, 3 * k))"), Seq(Row(Map(50 -> true, 78 -> false)))) - checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * k, 3 * k))"), + checkAnswer(dfExample3.select(transform_keys(col("x"), + (k, v) => when(v, k * 2).otherwise(k * 3))), Seq(Row(Map(50 -> true, 78 -> false)))) checkAnswer(dfExample4.selectExpr("transform_keys(y, (k, v) -> array_contains(k, 3) AND v)"), Seq(Row(Map(false -> false)))) + + checkAnswer(dfExample4.select(transform_keys(col("y"), + (k, v) => array_contains(k, lit(3)) && v)), + Seq(Row(Map(false -> false)))) } // Test with local relation, the Project will be evaluated without codegen @@ -2702,6 +3025,11 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } assert(ex3.getMessage.contains("Cannot use null as map key")) + val ex3a = intercept[Exception] { + dfExample1.select(transform_keys(col("i"), (k, v) => v)).show() + } + assert(ex3a.getMessage.contains("Cannot use null as map key")) + val ex4 = intercept[AnalysisException] { dfExample2.selectExpr("transform_keys(j, (k, v) -> k + 1)") } @@ -2766,6 +3094,46 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer( dfExample5.selectExpr("transform_values(c, (k, v) -> k + cardinality(v))"), Seq(Row(Map(1 -> 3)))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => k + v)), + Seq(Row(Map(1 -> 2, 9 -> 18, 8 -> 16, 7 -> 14)))) + + checkAnswer(dfExample2.select( + transform_values(col("x"), (k, v) => when(k, v).otherwise(k.cast("string")))), + Seq(Row(Map(false -> "false", true -> "def")))) + + checkAnswer(dfExample2.select(transform_values(col("x"), + (k, v) => (!k) && v === "abc")), + Seq(Row(Map(false -> true, true -> false)))) + + checkAnswer(dfExample3.select(transform_values(col("y"), (k, v) => v * v)), + Seq(Row(Map("a" -> 1, "b" -> 4, "c" -> 9)))) + + checkAnswer(dfExample3.select( + transform_values(col("y"), (k, v) => concat(k, lit(":"), v.cast("string")))), + Seq(Row(Map("a" -> "a:1", "b" -> "b:2", "c" -> "c:3")))) + + checkAnswer( + dfExample3.select(transform_values(col("y"), (k, v) => concat(k, v.cast("string")))), + Seq(Row(Map("a" -> "a1", "b" -> "b2", "c" -> "c3")))) + + val testMap = map_from_arrays( + array(lit(1), lit(2), lit(3)), + array(lit("one"), lit("two"), lit("three")) + ) + + checkAnswer( + dfExample4.select(transform_values(col("z"), + (k, v) => concat(element_at(testMap, k), lit("_"), v.cast("string")))), + Seq(Row(Map(1 -> "one_1.0", 2 -> "two_1.4", 3 ->"three_1.7")))) + + checkAnswer( + dfExample4.select(transform_values(col("z"), (k, v) => k - v)), + Seq(Row(Map(1 -> 0.0, 2 -> 0.6000000000000001, 3 -> 1.3)))) + + checkAnswer( + dfExample5.select(transform_values(col("c"), (k, v) => k + size(v))), + Seq(Row(Map(1 -> 3)))) } // Test with local relation, the Project will be evaluated without codegen @@ -2809,6 +3177,28 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(dfExample2.selectExpr("transform_values(j, (k, v) -> k + cast(v as BIGINT))"), Seq(Row(Map.empty[BigInt, BigInt]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), + (k, v) => lit(null).cast("int"))), + Seq(Row(Map.empty[Integer, Integer]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => k)), + Seq(Row(Map.empty[Integer, Integer]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => v)), + Seq(Row(Map.empty[Integer, Integer]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => lit(0))), + Seq(Row(Map.empty[Integer, Integer]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => lit("value"))), + Seq(Row(Map.empty[Integer, String]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => lit(true))), + Seq(Row(Map.empty[Integer, Boolean]))) + + checkAnswer(dfExample1.select(transform_values(col("i"), (k, v) => v.cast("bigint"))), + Seq(Row(Map.empty[BigInt, BigInt]))) } testEmpty() @@ -2833,6 +3223,15 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(dfExample2.selectExpr( "transform_values(b, (k, v) -> IF(v IS NULL, k + 1, k + 2))"), Seq(Row(Map(1 -> 3, 2 -> 4, 3 -> 4)))) + + checkAnswer(dfExample1.select(transform_values(col("a"), + (k, v) => lit(null).cast("int"))), + Seq(Row(Map[Int, Integer](1 -> null, 2 -> null, 3 -> null, 4 -> null)))) + + checkAnswer(dfExample2.select( + transform_values(col("b"), (k, v) => when(v.isNull, k + 1).otherwise(k + 2)) + ), + Seq(Row(Map(1 -> 3, 2 -> 4, 3 -> 4)))) } testNullValue() @@ -2871,6 +3270,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } assert(ex3.getMessage.contains( "data type mismatch: argument 1 requires map type")) + + val ex3a = intercept[AnalysisException] { + dfExample3.select(transform_values(col("x"), (k, v) => k + 1)) + } + assert(ex3a.getMessage.contains( + "data type mismatch: argument 1 requires map type")) } testInvalidLambdaFunctions() @@ -2897,10 +3302,15 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq.empty), Row(null)) checkAnswer(df1.selectExpr("zip_with(val1, val2, (x, y) -> x + y)"), expectedValue1) + checkAnswer(df1.select(zip_with(df1("val1"), df1("val2"), (x, y) => x + y)), expectedValue1) val expectedValue2 = Seq( Row(Seq(Row(1L, 1), Row(2L, null), Row(null, 3))), Row(Seq(Row(4L, 1), Row(11L, 2), Row(null, 3)))) checkAnswer(df2.selectExpr("zip_with(val1, val2, (x, y) -> (y, x))"), expectedValue2) + checkAnswer( + df2.select(zip_with(df2("val1"), df2("val2"), (x, y) => struct(y, x))), + expectedValue2 + ) } test("arrays zip_with function - for non-primitive types") { @@ -2915,7 +3325,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(Seq(Row("x", "a"), Row("y", null))), Row(Seq.empty), Row(null)) - checkAnswer(df.selectExpr("zip_with(val1, val2, (x, y) -> (y, x))"), expectedValue1) + checkAnswer( + df.selectExpr("zip_with(val1, val2, (x, y) -> (y, x))"), + expectedValue1 + ) + checkAnswer( + df.select(zip_with(col("val1"), col("val2"), (x, y) => struct(y, x))), + expectedValue1 + ) } test("arrays zip_with function - invalid") { @@ -2937,6 +3354,10 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { df.selectExpr("zip_with(i, a2, (acc, x) -> x)") } assert(ex3.getMessage.contains("data type mismatch: argument 1 requires array type")) + val ex3a = intercept[AnalysisException] { + df.select(zip_with(df("i"), df("a2"), (acc, x) => x)) + } + assert(ex3a.getMessage.contains("data type mismatch: argument 1 requires array type")) val ex4 = intercept[AnalysisException] { df.selectExpr("zip_with(a1, a, (acc, x) -> x)") } 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 2306c3ff039f1..ffd26816bde8f 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 @@ -21,6 +21,7 @@ import java.io.{ByteArrayOutputStream, File} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.util.UUID +import java.util.concurrent.atomic.AtomicLong import scala.util.Random @@ -553,7 +554,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { } test("replace column using withColumns") { - val df2 = sparkContext.parallelize(Array((1, 2), (2, 3), (3, 4))).toDF("x", "y") + val df2 = sparkContext.parallelize(Seq((1, 2), (2, 3), (3, 4))).toDF("x", "y") val df3 = df2.withColumns(Seq("x", "newCol1", "newCol2"), Seq(df2("x") + 1, df2("y"), df2("y") + 1)) checkAnswer( @@ -2105,17 +2106,17 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { // partitions. .write.partitionBy("p").option("compression", "gzip").json(path.getCanonicalPath) - var numJobs = 0 + val numJobs = new AtomicLong(0) sparkContext.addSparkListener(new SparkListener { override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { - numJobs += 1 + numJobs.incrementAndGet() } }) val df = spark.read.json(path.getCanonicalPath) assert(df.columns === Array("i", "p")) spark.sparkContext.listenerBus.waitUntilEmpty() - assert(numJobs == 1) + assert(numJobs.get() == 1L) } } @@ -2202,4 +2203,12 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { |*(1) Range (0, 10, step=1, splits=2)""".stripMargin)) } } + + test("SPARK-29442 Set `default` mode should override the existing mode") { + val df = Seq(Tuple1(1)).toDF() + val writer = df.write.mode("overwrite").mode("default") + val modeField = classOf[DataFrameWriter[Tuple1[Int]]].getDeclaredField("mode") + modeField.setAccessible(true) + assert(SaveMode.ErrorIfExists === modeField.get(writer).asInstanceOf[SaveMode]) + } } 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 dba906f63aed4..e47a6a68a0a9c 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 @@ -55,7 +55,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.map(func) i += 1 } - res.foreach(_ => Unit) + res.foreach(_ => ()) } benchmark.addCase("DataFrame") { iter => @@ -65,7 +65,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.select($"l" + 1 as "l") i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("Dataset") { iter => @@ -75,7 +75,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.map(func) i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark @@ -96,7 +96,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.map(func) i += 1 } - res.foreach(_ => Unit) + res.foreach(_ => ()) } benchmark.addCase("DataFrame") { iter => @@ -106,7 +106,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.select($"l" + 1 as "l", $"s") i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("Dataset") { iter => @@ -116,7 +116,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.map(func) i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark @@ -139,7 +139,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.filter(func) i += 1 } - res.foreach(_ => Unit) + res.foreach(_ => ()) } benchmark.addCase("DataFrame") { iter => @@ -149,7 +149,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.filter($"l" % 2L === 0L) i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("Dataset") { iter => @@ -159,7 +159,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.filter(func) i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark @@ -183,7 +183,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.filter(funcs(i)) i += 1 } - res.foreach(_ => Unit) + res.foreach(_ => ()) } benchmark.addCase("DataFrame") { iter => @@ -193,7 +193,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.filter($"l" % (100L + i) === 0L) i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("Dataset") { iter => @@ -203,7 +203,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { res = res.filter(funcs(i)) i += 1 } - res.queryExecution.toRdd.foreach(_ => Unit) + res.queryExecution.toRdd.foreach(_ => ()) } benchmark @@ -235,15 +235,15 @@ object DatasetBenchmark extends SqlBasedBenchmark { } benchmark.addCase("DataFrame sum") { iter => - df.select(sum($"l")).queryExecution.toRdd.foreach(_ => Unit) + df.select(sum($"l")).queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("Dataset sum using Aggregator") { iter => - df.as[Data].select(typed.sumLong((d: Data) => d.l)).queryExecution.toRdd.foreach(_ => Unit) + df.as[Data].select(typed.sumLong((d: Data) => d.l)).queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("Dataset complex Aggregator") { iter => - df.as[Data].select(ComplexAggregator.toColumn).queryExecution.toRdd.foreach(_ => Unit) + df.as[Data].select(ComplexAggregator.toColumn).queryExecution.toRdd.foreach(_ => ()) } benchmark 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 42e5ee58954e8..9c50e374f74de 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 @@ -21,6 +21,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import java.sql.{Date, Timestamp} import org.scalatest.exceptions.TestFailedException +import org.scalatest.prop.TableDrivenPropertyChecks._ import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.sql.catalyst.ScroogeLikeExample @@ -1841,6 +1842,24 @@ class DatasetSuite extends QueryTest with SharedSparkSession { val instant = java.time.Instant.parse("2019-03-30T09:54:00Z") assert(spark.range(1).map { _ => instant }.head === instant) } + + val dotColumnTestModes = Table( + ("caseSensitive", "colName"), + ("true", "field.1"), + ("false", "Field.1") + ) + + test("SPARK-25153: Improve error messages for columns with dots/periods") { + forAll(dotColumnTestModes) { (caseSensitive, colName) => + val ds = Seq(SpecialCharClass("1", "2")).toDS + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + val errorMsg = intercept[AnalysisException] { + ds(colName) + } + assert(errorMsg.getMessage.contains(s"did you mean to quote the `$colName` column?")) + } + } + } } object AssertExecutionId { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 09221efe28e15..e522aa1809d29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.plans.ExistenceJoin import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions._ @@ -161,22 +162,36 @@ class DynamicPartitionPruningSuite df: DataFrame, withSubquery: Boolean, withBroadcast: Boolean): Unit = { - val dpExprs = collectDynamicPruningExpressions(df.queryExecution.executedPlan) + val plan = df.queryExecution.executedPlan + val dpExprs = collectDynamicPruningExpressions(plan) val hasSubquery = dpExprs.exists { case InSubqueryExec(_, _: SubqueryExec, _, _) => true case _ => false } - val hasSubqueryBroadcast = dpExprs.exists { - case InSubqueryExec(_, _: SubqueryBroadcastExec, _, _) => true - case _ => false + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _) => b } val hasFilter = if (withSubquery) "Should" else "Shouldn't" assert(hasSubquery == withSubquery, s"$hasFilter trigger DPP with a subquery duplicate:\n${df.queryExecution}") val hasBroadcast = if (withBroadcast) "Should" else "Shouldn't" - assert(hasSubqueryBroadcast == withBroadcast, + assert(subqueryBroadcast.nonEmpty == withBroadcast, s"$hasBroadcast trigger DPP with a reused broadcast exchange:\n${df.queryExecution}") + + subqueryBroadcast.foreach { s => + s.child match { + case _: ReusedExchangeExec => // reuse check ok. + case b: BroadcastExchangeExec => + val hasReuse = plan.find { + case ReusedExchangeExec(_, e) => e eq b + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case _ => + fail(s"Invalid child node found in\n$s") + } + } } /** 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 125cff0e6628a..e70aeae9e605e 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.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -36,6 +37,19 @@ class ExplainSuite extends QueryTest with SharedSparkSession { f(normalizedOutput) } + /** + * Get the explain by running the sql. The explain mode should be part of the + * sql text itself. + */ + private def withNormalizedExplain(queryText: String)(f: String => Unit) = { + val output = new java.io.ByteArrayOutputStream() + Console.withOut(output) { + sql(queryText).show(false) + } + val normalizedOutput = output.toString.replaceAll("#\\d+", "#x") + f(normalizedOutput) + } + /** * Runs the plan and makes sure the plans contains all of the keywords. */ @@ -81,7 +95,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { // plan should show the rewritten aggregate expression. val df = sql("SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k") checkKeywordsExistsInExplain(df, - "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS some(v)#x, " + + "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS any(v)#x, " + "max(v#x) AS any(v)#x]") } } @@ -200,6 +214,54 @@ class ExplainSuite extends QueryTest with SharedSparkSession { } } } + + test("explain formatted - check presence of subquery in case of DPP") { + withTable("df1", "df2") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + withTable("df1", "df2") { + spark.range(1000).select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format("parquet") + .mode("overwrite") + .saveAsTable("df1") + + spark.range(100) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format("parquet") + .mode("overwrite") + .saveAsTable("df2") + + val sqlText = + """ + |EXPLAIN FORMATTED SELECT df1.id, df2.k + |FROM df1 JOIN df2 ON df1.k = df2.k AND df2.id < 2 + |""".stripMargin + + val expected_pattern1 = + "Subquery:1 Hosting operator id = 1 Hosting Expression = k#xL IN subquery#x" + val expected_pattern2 = + "PartitionFilters: \\[isnotnull\\(k#xL\\), dynamicpruningexpression\\(k#xL " + + "IN subquery#x\\)\\]" + val expected_pattern3 = + "Location: PrunedInMemoryFileIndex \\[.*org.apache.spark.sql.ExplainSuite" + + "/df2/.*, ... 99 entries\\]" + val expected_pattern4 = + "Location: PrunedInMemoryFileIndex \\[.*org.apache.spark.sql.ExplainSuite" + + "/df1/.*, ... 999 entries\\]" + withNormalizedExplain(sqlText) { normalizedOutput => + assert(expected_pattern1.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_pattern2.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_pattern3.r.findAllMatchIn(normalizedOutput).length == 2) + assert(expected_pattern4.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 e4c10d1990f25..d08f4b9066d2b 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 @@ -481,7 +481,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { spark.range(1000).repartition(1).write.csv(path) val bytesReads = new mutable.ArrayBuffer[Long]() val bytesReadListener = new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead } } 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 72742644ff34e..62f2d21e52704 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 @@ -22,12 +22,14 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer +import org.mockito.Mockito._ + 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.catalyst.plans.logical.Filter -import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec} +import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf @@ -37,6 +39,23 @@ import org.apache.spark.sql.types.StructType class JoinSuite extends QueryTest with SharedSparkSession { import testImplicits._ + private def attachCleanupResourceChecker(plan: SparkPlan): Unit = { + // SPARK-21492: Check cleanupResources are finally triggered in SortExec node for every + // test case + plan.foreachUp { + case s: SortExec => + val sortExec = spy(s) + verify(sortExec, atLeastOnce).cleanupResources() + verify(sortExec.rowSorter, atLeastOnce).cleanupResources() + case _ => + } + } + + override protected def checkAnswer(df: => DataFrame, rows: Seq[Row]): Unit = { + attachCleanupResourceChecker(df.queryExecution.sparkPlan) + super.checkAnswer(df, rows) + } + setupTestData() def statisticSizeInByte(df: DataFrame): BigInt = { @@ -1039,4 +1058,16 @@ class JoinSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(1, 2, 1, 2) :: Nil) } + + test("SPARK-21492: cleanupResource without code generation") { + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df1 = spark.range(0, 10, 1, 2) + val df2 = spark.range(10).select($"id".as("b1"), (- $"id").as("b2")) + val res = df1.join(df2, $"id" === $"b1" && $"id" === $"b2").select($"b1", $"b2", $"id") + checkAnswer(res, Row(0, 0, 0)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala index 6b90f20a94fa4..36db95ff8a31b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala @@ -27,14 +27,14 @@ trait LocalSparkSession extends BeforeAndAfterEach with BeforeAndAfterAll { self @transient var spark: SparkSession = _ - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE) SparkSession.clearActiveSession() SparkSession.clearDefaultSession() } - override def afterEach() { + override def afterEach(): Unit = { try { LocalSparkSession.stop(spark) SparkSession.clearActiveSession() @@ -47,7 +47,7 @@ trait LocalSparkSession extends BeforeAndAfterEach with BeforeAndAfterAll { self } object LocalSparkSession { - def stop(spark: SparkSession) { + def stop(spark: SparkSession): Unit = { if (spark != null) { spark.stop() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala new file mode 100644 index 0000000000000..1354dcfda45fe --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala @@ -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.spark.sql + +import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession { + + override def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) + + test("cast string to boolean") { + Seq("true", "tru", "tr", "t", " tRue ", " tRu ", "yes", "ye", + "y", "1", "on").foreach { input => + checkAnswer(sql(s"select cast('$input' as boolean)"), Row(true)) + } + Seq("false", "fals", "fal", "fa", "f", " fAlse ", " fAls ", "no", "n", + "0", "off", "of").foreach { input => + checkAnswer(sql(s"select cast('$input' as boolean)"), Row(false)) + } + + Seq("o", "abc", "").foreach { input => + checkAnswer(sql(s"select cast('$input' as boolean)"), Row(null)) + } + } +} 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 28a027690db04..630489ad9c603 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 @@ -22,11 +22,16 @@ import java.net.{MalformedURLException, URL} import java.sql.{Date, Timestamp} import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.parallel.immutable.ParVector + import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.command.FunctionsCommand 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 @@ -55,7 +60,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { test("show functions") { def getFunctions(pattern: String): Seq[Row] = { StringUtils.filterPattern( - spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern) + spark.sessionState.catalog.listFunctions("default").map(_._1.funcName) + ++ FunctionsCommand.virtualOperators, pattern) .map(Row(_)) } @@ -115,6 +121,83 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { } } + test("using _FUNC_ instead of function names in examples") { + val exampleRe = "(>.*;)".r + val setStmtRe = "(?i)^(>\\s+set\\s+).+".r + val ignoreSet = Set( + // Examples for CaseWhen show simpler syntax: + // `CASE WHEN ... THEN ... WHEN ... THEN ... END` + "org.apache.spark.sql.catalyst.expressions.CaseWhen", + // _FUNC_ is replaced by `locate` but `locate(... IN ...)` is not supported + "org.apache.spark.sql.catalyst.expressions.StringLocate", + // _FUNC_ is replaced by `%` which causes a parsing error on `SELECT %(2, 1.8)` + "org.apache.spark.sql.catalyst.expressions.Remainder", + // Examples demonstrate alternative names, see SPARK-20749 + "org.apache.spark.sql.catalyst.expressions.Length") + spark.sessionState.functionRegistry.listFunction().foreach { funcId => + val info = spark.sessionState.catalog.lookupFunctionInfo(funcId) + val className = info.getClassName + withClue(s"Expression class '$className'") { + val exprExamples = info.getOriginalExamples + if (!exprExamples.isEmpty && !ignoreSet.contains(className)) { + assert(exampleRe.findAllIn(exprExamples).toIterable + .filter(setStmtRe.findFirstIn(_).isEmpty) // Ignore SET commands + .forall(_.contains("_FUNC_"))) + } + } + } + } + + test("check outputs of expression examples") { + def unindentAndTrim(s: String): String = { + s.replaceAll("\n\\s+", "\n").trim + } + val beginSqlStmtRe = " > ".r + val endSqlStmtRe = ";\n".r + def checkExampleSyntax(example: String): Unit = { + val beginStmtNum = beginSqlStmtRe.findAllIn(example).length + val endStmtNum = endSqlStmtRe.findAllIn(example).length + assert(beginStmtNum === endStmtNum, + "The number of ` > ` does not match to the number of `;`") + } + val exampleRe = """^(.+);\n(?s)(.+)$""".r + val ignoreSet = Set( + // One of examples shows getting the current timestamp + "org.apache.spark.sql.catalyst.expressions.UnixTimestamp", + // Random output without a seed + "org.apache.spark.sql.catalyst.expressions.Rand", + "org.apache.spark.sql.catalyst.expressions.Randn", + "org.apache.spark.sql.catalyst.expressions.Shuffle", + "org.apache.spark.sql.catalyst.expressions.Uuid", + // The example calls methods that return unstable results. + "org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection") + + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + val parFuncs = new ParVector(spark.sessionState.functionRegistry.listFunction().toVector) + parFuncs.foreach { funcId => + // Examples can change settings. We clone the session to prevent tests clashing. + val clonedSpark = spark.cloneSession() + val info = clonedSpark.sessionState.catalog.lookupFunctionInfo(funcId) + val className = info.getClassName + if (!ignoreSet.contains(className)) { + withClue(s"Function '${info.getName}', Expression class '$className'") { + val example = info.getExamples + checkExampleSyntax(example) + example.split(" > ").toList.foreach(_ match { + case exampleRe(sql, output) => + val df = clonedSpark.sql(sql) + val actual = unindentAndTrim( + hiveResultString(df.queryExecution.executedPlan).mkString("\n")) + val expected = unindentAndTrim(output) + assert(actual === expected) + case _ => + }) + } + } + } + } + } + test("SPARK-6743: no columns from cache") { Seq( (83, 0, 38), @@ -3149,6 +3232,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { 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)) } + sql("DROP VIEW t1") } test("SPARK-28156: self-join should not miss cached view") { @@ -3192,6 +3276,47 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { checkAnswer(df3, Array(Row(new java.math.BigDecimal("0.100000000000000000000000100")))) } } + + test("SPARK-29239: Subquery should not cause NPE when eliminating subexpression") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> "CODEGEN_ONLY", + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> ConvertToLocalRelation.ruleName) { + withTempView("t1", "t2") { + sql("create temporary view t1 as select * from values ('val1a', 10L) as t1(t1a, t1b)") + sql("create temporary view t2 as select * from values ('val3a', 110L) as t2(t2a, t2b)") + val df = sql("SELECT min, min from (SELECT (SELECT min(t2b) FROM t2) min " + + "FROM t1 WHERE t1a = 'val1c')") + assert(df.collect().size == 0) + } + } + } + + test("SPARK-29213: FilterExec should not throw NPE") { + withTempView("t1", "t2", "t3") { + sql("SELECT ''").as[String].map(identity).toDF("x").createOrReplaceTempView("t1") + sql("SELECT * FROM VALUES 0, CAST(NULL AS BIGINT)") + .as[java.lang.Long] + .map(identity) + .toDF("x") + .createOrReplaceTempView("t2") + sql("SELECT ''").as[String].map(identity).toDF("x").createOrReplaceTempView("t3") + sql( + """ + |SELECT t1.x + |FROM t1 + |LEFT JOIN ( + | SELECT x FROM ( + | SELECT x FROM t2 + | UNION ALL + | SELECT SUBSTR(x,5) x FROM t3 + | ) a + | WHERE LENGTH(x)>0 + |) t3 + |ON t1.x=t3.x + """.stripMargin).collect() + } + } } 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 1a41dd95a5700..58469fca8f3fe 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 @@ -22,10 +22,9 @@ import java.util.{Locale, TimeZone} import scala.util.control.NonFatal -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.execution.HiveResult.hiveResultString @@ -135,9 +134,27 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { private val notIncludedMsg = "[not included in comparison]" private val clsName = this.getClass.getCanonicalName + protected val emptySchema = StructType(Seq.empty).catalogString + + protected override def sparkConf: SparkConf = super.sparkConf + // Fewer shuffle partitions to speed up testing. + .set(SQLConf.SHUFFLE_PARTITIONS, 4) + /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( - "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. + "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. + // SPARK-28885 String value is not allowed to be stored as numeric type with + // ANSI store assignment policy. + "postgreSQL/numeric.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/float4.sql", + "postgreSQL/float8.sql", + // SPARK-28885 String value is not allowed to be stored as date/timestamp type with + // ANSI store assignment policy. + "postgreSQL/date.sql", + "postgreSQL/timestamp.sql" ) // Create all the test cases. @@ -293,10 +310,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { testCase match { case udfTestCase: UDFTest => - // In Python UDF tests, the number of shuffle partitions matters considerably in - // the testing time because it requires to fork and communicate between external - // processes. - localSparkSession.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, 4) registerTestUDF(udfTestCase.udf, localSparkSession) case _ => } @@ -311,8 +324,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // PostgreSQL enabled cartesian product by default. localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) - localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _ => } @@ -324,11 +336,11 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } // Run the SQL queries preparing them for comparison. val outputs: Seq[QueryOutput] = queries.map { sql => - val (schema, output) = getNormalizedResult(localSparkSession, sql) + val (schema, output) = handleExceptions(getNormalizedResult(localSparkSession, sql)) // We might need to do some query canonicalization in the future. QueryOutput( sql = sql, - schema = schema.catalogString, + schema = schema, output = output.mkString("\n").replaceAll("\\s+$", "")) } @@ -389,54 +401,63 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } } - /** Executes a query and returns the result as (schema of the output, normalized output). */ - private def getNormalizedResult(session: SparkSession, sql: String): (StructType, Seq[String]) = { - // 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 _: DescribeCommandBase - | _: DescribeColumnCommand - | _: DescribeTableStatement - | _: DescribeColumnStatement => true - case PhysicalOperation(_, _, Sort(_, true, _)) => true - case _ => plan.children.iterator.exists(isSorted) - } - + /** + * This method handles exceptions occurred during query execution as they may need special care + * to become comparable to the expected output. + * + * @param result a function that returns a pair of schema and output + */ + protected def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { try { - val df = session.sql(sql) - val schema = df.schema - // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = SQLExecution.withNewExecutionId(session, df.queryExecution, Some(sql)) { - hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) - } - - // If the output is not pre-sorted, sort it. - if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) - + result } catch { case a: AnalysisException => // Do not output the logical plan tree which contains expression IDs. // Also implement a crude way of masking expression IDs in the error message // with a generic pattern "###". val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage - (StructType(Seq.empty), Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) + (emptySchema, Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) case s: SparkException if s.getCause != null => // For a runtime exception, it is hard to match because its message contains // information of stage, task ID, etc. // To make result matching simpler, here we match the cause of the exception if it exists. val cause = s.getCause - (StructType(Seq.empty), Seq(cause.getClass.getName, cause.getMessage)) + (emptySchema, Seq(cause.getClass.getName, cause.getMessage)) case NonFatal(e) => // If there is an exception, put the exception class followed by the message. - (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) + (emptySchema, Seq(e.getClass.getName, e.getMessage)) + } + } + + /** Executes a query and returns the result as (schema of the output, normalized output). */ + private def getNormalizedResult(session: SparkSession, sql: String): (String, Seq[String]) = { + // 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 _: DescribeCommandBase + | _: DescribeColumnCommand + | _: DescribeTableStatement + | _: DescribeColumnStatement => true + case PhysicalOperation(_, _, Sort(_, true, _)) => true + case _ => plan.children.iterator.exists(isSorted) + } + + val df = session.sql(sql) + val schema = df.schema.catalogString + // Get answer, but also get rid of the #1234 expression ids that show up in explain plans + val answer = SQLExecution.withNewExecutionId(session, df.queryExecution, Some(sql)) { + hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) } + + // If the output is not pre-sorted, sort it. + if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) } protected def replaceNotIncludedMsg(line: String): String = { line.replaceAll("#\\d+", "#x") .replaceAll( - s"Location.*/sql/core/spark-warehouse/$clsName/", - s"Location ${notIncludedMsg}sql/core/spark-warehouse/") + s"Location.*$clsName/", + s"Location ${notIncludedMsg}/{warehouse_dir}/") .replaceAll("Created By.*", s"Created By $notIncludedMsg") .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") @@ -451,7 +472,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) if (file.getAbsolutePath.startsWith( - s"$inputFilePath${File.separator}udf${File.separator}pgSQL")) { + s"$inputFilePath${File.separator}udf${File.separator}postgreSQL")) { Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => UDFPgSQLTestCase( s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) @@ -461,7 +482,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { UDFTestCase( s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) } - } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil } else { RegularTestCase(testCaseName, absPath, resultFile) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala index 9a0c61b3304c5..099b559105fe8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SSBQuerySuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.resourceToString */ class SSBQuerySuite extends BenchmarkQueryTest { - override def beforeAll { + override def beforeAll: Unit = { super.beforeAll sql( 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 74341f93dd5ba..b8df6f2bebf53 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,13 +16,15 @@ */ package org.apache.spark.sql +import java.util.Locale + 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._ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, UnresolvedHint} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector @@ -122,6 +124,25 @@ class SparkSessionExtensionSuite extends SparkFunSuite { } } + case class MyHintRule(spark: SparkSession) extends Rule[LogicalPlan] { + val MY_HINT_NAME = Set("CONVERT_TO_EMPTY") + + override def apply(plan: LogicalPlan): LogicalPlan = + plan.resolveOperators { + case h: UnresolvedHint if MY_HINT_NAME.contains(h.name.toUpperCase(Locale.ROOT)) => + LocalRelation(h.output, data = Seq.empty, isStreaming = h.isStreaming) + } + } + + test("inject custom hint rule") { + withSession(Seq(_.injectPostHocResolutionRule(MyHintRule))) { session => + assert( + session.range(1).hint("CONVERT_TO_EMPTY").logicalPlan.isInstanceOf[LocalRelation], + "plan is expected to be a local relation" + ) + } + } + test("inject columnar") { val extensions = create { extensions => extensions.injectColumnar(session => 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 5049df3219959..ec698818a0d85 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 @@ -285,7 +285,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { test("string parse_url function") { - def testUrl(url: String, expected: Row) { + def testUrl(url: String, expected: Row): Unit = { checkAnswer(Seq[String]((url)).toDF("url").selectExpr( "parse_url(url, 'HOST')", "parse_url(url, 'PATH')", "parse_url(url, 'QUERY')", "parse_url(url, 'REF')", 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 a1d7792941ed9..266f8e23712db 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 @@ -204,6 +204,154 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } + test("SPARK-29145: JOIN Condition use QueryList") { + withTempView("s1", "s2", "s3") { + Seq(1, 3, 5, 7, 9).toDF("id").createOrReplaceTempView("s1") + Seq(1, 3, 4, 6, 9).toDF("id").createOrReplaceTempView("s2") + Seq(3, 4, 6, 9).toDF("id").createOrReplaceTempView("s3") + + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id IN (SELECT 9) + """.stripMargin), + Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id NOT IN (SELECT 9) + """.stripMargin), + Row(1) :: Row(3) :: Nil) + + // case `IN` + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(3) :: Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id AS id2 FROM s1 + | LEFT SEMI JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(3) :: Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id as id2 FROM s1 + | LEFT ANTI JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(1) :: Row(5) :: Row(7) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id as id2 FROM s1 + | LEFT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(1, null) :: Row(3, 3) :: Row(5, null) :: Row(7, null) :: Row(9, 9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id as id2 FROM s1 + | RIGHT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(null, 1) :: Row(3, 3) :: Row(null, 4) :: Row(null, 6) :: Row(9, 9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | FULL OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(1, null) :: Row(3, 3) :: Row(5, null) :: Row(7, null) :: Row(9, 9) :: + Row(null, 1) :: Row(null, 4) :: Row(null, 6) :: Nil) + + // case `NOT IN` + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id AS id2 FROM s1 + | LEFT SEMI JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id AS id2 FROM s1 + | LEFT ANTI JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(3) :: Row(5) :: Row(7) :: Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | LEFT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1, 1) :: Row(3, null) :: Row(5, null) :: Row(7, null) :: Row(9, null) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | RIGHT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1, 1) :: Row(null, 3) :: Row(null, 4) :: Row(null, 6) :: Row(null, 9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | FULL OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1, 1) :: Row(3, null) :: Row(5, null) :: Row(7, null) :: Row(9, null) :: + Row(null, 3) :: Row(null, 4) :: Row(null, 6) :: Row(null, 9) :: Nil) + } + } + test("SPARK-14791: scalar subquery inside broadcast join") { val df = sql("select a, sum(b) as s from l group by a having a > (select avg(a) from l)") val expected = Row(3, 2.0, 3, 3.0) :: Row(6, null, 6, null) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala index 11e66e878c149..aacb625d7921f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.internal.SQLConf */ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSSchema { - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() for (tableName <- tableNames) { createTable(spark, tableName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala index b32d95d0b286c..ba99e18714b1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.resourceToString */ class TPCHQuerySuite extends BenchmarkQueryTest { - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 03cefa0d2e77a..eed07aeff0903 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -50,7 +50,7 @@ trait AlterTableTests extends SharedSparkSession { } assert(exc.getMessage.contains(s"${catalogAndNamespace}table_name")) - assert(exc.getMessage.contains("Table or view not found")) + assert(exc.getMessage.contains("Table not found")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 207ece83759ed..08627e681f9e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -25,8 +25,9 @@ import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG +import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -83,7 +84,7 @@ class DataSourceV2DataFrameSessionCatalogSuite val t1 = "prop_table" withTable(t1) { spark.range(20).write.format(v2Format).option("path", "abc").saveAsTable(t1) - val cat = spark.sessionState.catalogManager.v2SessionCatalog.asInstanceOf[TableCatalog] + val cat = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] val tableInfo = cat.loadTable(Identifier.of(Array.empty, t1)) assert(tableInfo.properties().get("location") === "abc") assert(tableInfo.properties().get("provider") === v2Format) @@ -144,13 +145,13 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio protected val catalogClassName: String = classOf[InMemoryTableSessionCatalog].getName before { - spark.conf.set(V2_SESSION_CATALOG.key, catalogClassName) + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION.key, catalogClassName) } override def afterEach(): Unit = { super.afterEach() - catalog("session").asInstanceOf[Catalog].clearTables() - spark.conf.unset(V2_SESSION_CATALOG.key) + catalog(SESSION_CATALOG_NAME).asInstanceOf[Catalog].clearTables() + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) } protected def verifyTable(tableName: String, expected: DataFrame): Unit diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index cd811bb7afb51..8e921aaeb66f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.{DataFrame, Row, SaveMode} +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException class DataSourceV2DataFrameSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) { @@ -75,13 +76,15 @@ class DataSourceV2DataFrameSuite withTable(t1) { sql(s"CREATE TABLE $t1 (id bigint, data string) USING foo") val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") - // Default saveMode is append, therefore this doesn't throw a table already exists exception - df.write.saveAsTable(t1) + // Default saveMode is ErrorIfExists + intercept[TableAlreadyExistsException] { + df.write.saveAsTable(t1) + } + assert(spark.table(t1).count() === 0) + + // appends are by name not by position + df.select('data, 'id).write.mode("append").saveAsTable(t1) checkAnswer(spark.table(t1), df) - - // also appends are by name not by position - df.select('data, 'id).write.saveAsTable(t1) - checkAnswer(spark.table(t1), df.union(df)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index e1a5dbe3351ec..27725bcadbcd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -44,7 +44,7 @@ class DataSourceV2SQLSessionCatalogSuite } override def getTableMetadata(tableName: String): Table = { - val v2Catalog = spark.sessionState.catalogManager.v2SessionCatalog + val v2Catalog = spark.sessionState.catalogManager.currentCatalog val nameParts = spark.sessionState.sqlParser.parseMultipartIdentifier(tableName) v2Catalog.asInstanceOf[TableCatalog] .loadTable(Identifier.of(Array.empty, nameParts.last)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 9eb8e5b3b73fc..4f2c1af8f7b56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -20,10 +20,12 @@ package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG +import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION +import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -66,7 +68,8 @@ class DataSourceV2SQLSuite spark.conf.set( "spark.sql.catalog.testcat_atomic", classOf[StagingInMemoryTableCatalog].getName) spark.conf.set("spark.sql.catalog.testcat2", classOf[InMemoryTableCatalog].getName) - spark.conf.set(V2_SESSION_CATALOG.key, classOf[InMemoryTableSessionCatalog].getName) + spark.conf.set( + V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[InMemoryTableSessionCatalog].getName) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -148,7 +151,7 @@ class DataSourceV2SQLSuite test("CreateTable: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING $v2Source") - val testCatalog = catalog("session").asTableCatalog + val testCatalog = catalog(SESSION_CATALOG_NAME).asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "default.table_name") @@ -407,7 +410,7 @@ class DataSourceV2SQLSuite test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name USING $v2Source AS SELECT id, data FROM source") - val testCatalog = catalog("session").asTableCatalog + val testCatalog = catalog(SESSION_CATALOG_NAME).asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "default.table_name") @@ -509,7 +512,7 @@ class DataSourceV2SQLSuite test("CreateTableAsSelect: v2 session catalog can load v1 source table") { // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG.key) + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -519,7 +522,7 @@ class DataSourceV2SQLSuite checkAnswer(sql(s"TABLE default.table_name"), spark.table("source")) // The fact that the following line doesn't throw an exception means, the session catalog // can load the table. - val t = catalog("session").asTableCatalog + val t = catalog(SESSION_CATALOG_NAME).asTableCatalog .loadTable(Identifier.of(Array.empty, "table_name")) assert(t.isInstanceOf[V1Table], "V1 table wasn't returned as an unresolved table") } @@ -724,6 +727,23 @@ class DataSourceV2SQLSuite expectV2Catalog = false) } + test("ShowTables: change current catalog and namespace with USE statements") { + sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") + + // Initially, the v2 session catalog (current catalog) is used. + runShowTablesSql( + "SHOW TABLES", Seq(Row("", "source", true), Row("", "source2", true)), + expectV2Catalog = false) + + // Update the current catalog, and no table is matched since the current namespace is Array(). + sql("USE testcat") + runShowTablesSql("SHOW TABLES", Seq()) + + // Update the current namespace to match ns1.ns2.table. + sql("USE testcat.ns1.ns2") + runShowTablesSql("SHOW TABLES", Seq(Row("ns1.ns2", "table"))) + } + private def runShowTablesSql( sqlText: String, expected: Seq[Row], @@ -744,6 +764,32 @@ class DataSourceV2SQLSuite assert(expected === df.collect()) } + test("CreateNameSpace: basic tests") { + // Session catalog is used. + sql("CREATE NAMESPACE ns") + testShowNamespaces("SHOW NAMESPACES", Seq("default", "ns")) + + // V2 non-session catalog is used. + sql("CREATE NAMESPACE testcat.ns1.ns2") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2")) + + // TODO: Add tests for validating namespace metadata when DESCRIBE NAMESPACE is available. + } + + test("CreateNameSpace: test handling of 'IF NOT EXIST'") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") + + // The 'ns1' namespace already exists, so this should fail. + val exception = intercept[NamespaceAlreadyExistsException] { + sql("CREATE NAMESPACE testcat.ns1") + } + assert(exception.getMessage.contains("Namespace 'ns1' already exists")) + + // The following will be no-op since the namespace already exists. + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set("spark.sql.default.catalog", "testcat") @@ -780,11 +826,8 @@ class DataSourceV2SQLSuite test("ShowNamespaces: default v2 catalog is not set") { spark.sql("CREATE TABLE testcat.ns.table (id bigint) USING foo") - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES") - } - - assert(exception.getMessage.contains("No default v2 catalog is set")) + // The current catalog is resolved to a v2 session catalog. + testShowNamespaces("SHOW NAMESPACES", Seq("default")) } test("ShowNamespaces: default v2 catalog doesn't support namespace") { @@ -812,12 +855,28 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("does not support namespaces")) } - test("ShowNamespaces: no v2 catalog is available") { + test("ShowNamespaces: session catalog is used and namespace doesn't exist") { val exception = intercept[AnalysisException] { sql("SHOW NAMESPACES in dummy") } - assert(exception.getMessage.contains("No v2 catalog is available")) + assert(exception.getMessage.contains("Namespace 'dummy' not found")) + } + + test("ShowNamespaces: change catalog and namespace with USE statements") { + sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") + + // Initially, the current catalog is a v2 session catalog. + testShowNamespaces("SHOW NAMESPACES", Seq("default")) + + // Update the current catalog to 'testcat'. + sql("USE testcat") + testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) + + // Update the current namespace to 'ns1'. + sql("USE ns1") + // 'SHOW NAMESPACES' is not affected by the current namespace and lists root namespaces. + testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) } private def testShowNamespaces( @@ -830,9 +889,70 @@ class DataSourceV2SQLSuite assert(df.collect().map(_.getAs[String](0)).sorted === expected.sorted) } + test("Use: basic tests with USE statements") { + val catalogManager = spark.sessionState.catalogManager + + // Validate the initial current catalog and namespace. + assert(catalogManager.currentCatalog.name() == SESSION_CATALOG_NAME) + assert(catalogManager.currentNamespace === Array("default")) + + // The following implicitly creates namespaces. + sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") + sql("CREATE TABLE testcat2.ns2.ns2_2.table (id bigint) USING foo") + sql("CREATE TABLE testcat2.ns3.ns3_3.table (id bigint) USING foo") + sql("CREATE TABLE testcat2.testcat.table (id bigint) USING foo") + + // Catalog is resolved to 'testcat'. + sql("USE testcat.ns1.ns1_1") + assert(catalogManager.currentCatalog.name() == "testcat") + assert(catalogManager.currentNamespace === Array("ns1", "ns1_1")) + + // Catalog is resolved to 'testcat2'. + sql("USE testcat2.ns2.ns2_2") + assert(catalogManager.currentCatalog.name() == "testcat2") + assert(catalogManager.currentNamespace === Array("ns2", "ns2_2")) + + // Only the namespace is changed. + sql("USE ns3.ns3_3") + assert(catalogManager.currentCatalog.name() == "testcat2") + assert(catalogManager.currentNamespace === Array("ns3", "ns3_3")) + + // Only the namespace is changed (explicit). + sql("USE NAMESPACE testcat") + assert(catalogManager.currentCatalog.name() == "testcat2") + assert(catalogManager.currentNamespace === Array("testcat")) + + // Catalog is resolved to `testcat`. + sql("USE testcat") + assert(catalogManager.currentCatalog.name() == "testcat") + assert(catalogManager.currentNamespace === Array()) + } + + test("Use: set v2 catalog as a current catalog") { + val catalogManager = spark.sessionState.catalogManager + assert(catalogManager.currentCatalog.name() == SESSION_CATALOG_NAME) + + sql("USE testcat") + assert(catalogManager.currentCatalog.name() == "testcat") + } + + test("Use: v2 session catalog is used and namespace does not exist") { + val exception = intercept[NoSuchDatabaseException] { + sql("USE ns1") + } + assert(exception.getMessage.contains("Database 'ns1' not found")) + } + + test("Use: v2 catalog is used and namespace does not exist") { + // Namespaces are not required to exist for v2 catalogs. + sql("USE testcat.ns1.ns2") + val catalogManager = spark.sessionState.catalogManager + assert(catalogManager.currentNamespace === Array("ns1", "ns2")) + } + test("tableCreation: partition column case insensitive resolution") { val testCatalog = catalog("testcat").asTableCatalog - val sessionCatalog = catalog("session").asTableCatalog + val sessionCatalog = catalog(SESSION_CATALOG_NAME).asTableCatalog def checkPartitioning(cat: TableCatalog, partition: String): Unit = { val table = cat.loadTable(Identifier.of(Array.empty, "tbl")) @@ -874,19 +994,19 @@ class DataSourceV2SQLSuite val errorMsg = "Found duplicate column(s) in the table definition of `t`" Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE t ($c0 INT, $c1 INT) USING $v2Source", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE testcat.t ($c0 INT, $c1 INT) USING $v2Source", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE t ($c0 INT, $c1 INT) USING $v2Source", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE testcat.t ($c0 INT, $c1 INT) USING $v2Source", errorMsg ) @@ -898,19 +1018,19 @@ class DataSourceV2SQLSuite val errorMsg = "Found duplicate column(s) in the table definition of `t`" Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE t (d struct<$c0: INT, $c1: INT>) USING $v2Source", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE testcat.t (d struct<$c0: INT, $c1: INT>) USING $v2Source", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE t (d struct<$c0: INT, $c1: INT>) USING $v2Source", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE testcat.t (d struct<$c0: INT, $c1: INT>) USING $v2Source", errorMsg ) @@ -920,20 +1040,20 @@ class DataSourceV2SQLSuite test("tableCreation: bucket column names not in table definition") { val errorMsg = "Couldn't find column c in" - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE tbl (a int, b string) USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE testcat.tbl (a int, b string) USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE tbl (a int, b string) USING $v2Source " + "CLUSTERED BY (c) INTO 4 BUCKETS", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE testcat.tbl (a int, b string) USING $v2Source " + "CLUSTERED BY (c) INTO 4 BUCKETS", errorMsg @@ -944,19 +1064,19 @@ class DataSourceV2SQLSuite val errorMsg = "Found duplicate column(s) in the partitioning" Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE testcat.t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE testcat.t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", errorMsg ) @@ -968,22 +1088,22 @@ class DataSourceV2SQLSuite val errorMsg = "Found duplicate column(s) in the bucket definition" Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE TABLE testcat.t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", errorMsg ) - testCreateAnalysisError( + assertAnalysisError( s"CREATE OR REPLACE TABLE testcat.t ($c0 INT) USING $v2Source " + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", errorMsg @@ -992,6 +1112,27 @@ class DataSourceV2SQLSuite } } + test("REFRESH TABLE: v2 table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + val testCatalog = catalog("testcat").asTableCatalog.asInstanceOf[InMemoryTableCatalog] + val identifier = Identifier.of(Array("ns1", "ns2"), "tbl") + + assert(!testCatalog.isTableInvalidated(identifier)) + sql(s"REFRESH TABLE $t") + assert(testCatalog.isTableInvalidated(identifier)) + } + } + + test("REPLACE TABLE: v1 table") { + val e = intercept[AnalysisException] { + sql(s"CREATE OR REPLACE TABLE tbl (a int) USING ${classOf[SimpleScanSource].getName}") + } + assert(e.message.contains("REPLACE TABLE is only supported with v2 tables")) + } + test("DeleteFrom: basic - delete all") { val t = "testcat.ns1.ns2.tbl" withTable(t) { @@ -1049,7 +1190,116 @@ class DataSourceV2SQLSuite } } - private def testCreateAnalysisError(sqlStatement: String, expectedError: String): Unit = { + test("UPDATE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, name string, age int, p int) + |USING foo + |PARTITIONED BY (id, p) + """.stripMargin) + + // UPDATE non-existing table + assertAnalysisError( + "UPDATE dummy SET name='abc'", + "Table not found") + + // UPDATE non-existing column + assertAnalysisError( + s"UPDATE $t SET dummy='abc'", + "cannot resolve") + assertAnalysisError( + s"UPDATE $t SET name='abc' WHERE dummy=1", + "cannot resolve") + + // UPDATE is not implemented yet. + val e = intercept[UnsupportedOperationException] { + sql(s"UPDATE $t SET name='Robert', age=32 WHERE p=1") + } + assert(e.getMessage.contains("UPDATE TABLE is not supported temporarily")) + } + } + + test("ANALYZE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + testV1Command("ANALYZE TABLE", s"$t COMPUTE STATISTICS") + testV1Command("ANALYZE TABLE", s"$t COMPUTE STATISTICS FOR ALL COLUMNS") + } + } + + test("MSCK REPAIR TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + testV1Command("MSCK REPAIR TABLE", t) + } + } + + test("TRUNCATE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, data string) + |USING foo + |PARTITIONED BY (id) + """.stripMargin) + + testV1Command("TRUNCATE TABLE", t) + testV1Command("TRUNCATE TABLE", s"$t PARTITION(id='1')") + } + } + + test("SHOW PARTITIONS") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, data string) + |USING foo + |PARTITIONED BY (id) + """.stripMargin) + + testV1Command("SHOW PARTITIONS", t) + testV1Command("SHOW PARTITIONS", s"$t PARTITION(id='1')") + } + } + + test("CACHE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + testV1Command("CACHE TABLE", t) + + val e = intercept[AnalysisException] { + sql(s"CACHE LAZY TABLE $t") + } + assert(e.message.contains("CACHE TABLE is only supported with v1 tables")) + } + } + + test("UNCACHE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + testV1Command("UNCACHE TABLE", t) + testV1Command("UNCACHE TABLE", s"IF EXISTS $t") + } + } + + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { + val e = intercept[AnalysisException] { + sql(s"$sqlCommand $sqlParams") + } + assert(e.message.contains(s"$sqlCommand is only supported with v1 tables")) + } + + private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) }.getMessage diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 00a8b430d33c2..138bbc3f04f64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -225,8 +225,12 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { spark.read.format(cls.getName).option("path", path).load(), spark.range(10).select('id, -'id)) - // default save mode is append - spark.range(10).select('id as 'i, -'id as 'j).write.format(cls.getName) + // default save mode is ErrorIfExists + intercept[AnalysisException] { + spark.range(10).select('id as 'i, -'id as 'j).write.format(cls.getName) + .option("path", path).save() + } + spark.range(10).select('id as 'i, -'id as 'j).write.mode("append").format(cls.getName) .option("path", path).save() checkAnswer( spark.read.format(cls.getName).option("path", path).load(), @@ -281,7 +285,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { val numPartition = 6 spark.range(0, 10, 1, numPartition).select('id as 'i, -'id as 'j).write.format(cls.getName) - .option("path", path).save() + .mode("append").option("path", path).save() checkAnswer( spark.read.format(cls.getName).option("path", path).load(), spark.range(10).select('id, -'id)) @@ -368,7 +372,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { val format = classOf[SimpleWritableDataSource].getName val df = Seq((1L, 2L)).toDF("i", "j") - df.write.format(format).option("path", optionPath).save() + df.write.format(format).mode("append").option("path", optionPath).save() assert(!new File(sessionPath).exists) checkAnswer(spark.read.format(format).option("path", optionPath).load(), df) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala new file mode 100644 index 0000000000000..aaec6a9761d63 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Properties + +import org.apache.spark.{SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, If, SortArray, SparkPartitionID, SpecificInternalRow} +import org.apache.spark.sql.catalyst.expressions.aggregate.CollectSet +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{LongType, StringType, StructType} +import org.apache.spark.unsafe.types.UTF8String + +/** + * Test suite for [[AggregatingAccumulator]]. + */ +class AggregatingAccumulatorSuite + extends SparkFunSuite + with SharedSparkSession + with ExpressionEvalHelper { + private val a = 'a.long + private val b = 'b.string + private val c = 'c.double + private val inputAttributes = Seq(a, b, c) + private def str(s: String): UTF8String = UTF8String.fromString(s) + + test("empty aggregation") { + val acc1 = AggregatingAccumulator( + Seq(sum(a) + 1L as "sum_a", max(b) as "max_b", approxCountDistinct(c) as "acntd_c"), + inputAttributes) + val expectedSchema = new StructType() + .add("sum_a", "long") + .add("max_b", "string") + .add("acntd_c", "long", nullable = false) + assert(acc1.schema === expectedSchema) + + val accEmpty = acc1.copy() + val acc2 = acc1.copy() + + // Merge empty + acc1.merge(accEmpty) + assert(acc1.isZero) + + // No updates + assert(acc1.isZero) + checkResult(acc1.value, InternalRow(null, null, 0), expectedSchema, false) + assert(acc1.isZero) + + // A few updates + acc1.add(InternalRow(4L, str("foo"), 4.9d)) + acc1.add(InternalRow(98L, str("bar"), -323.9d)) + acc1.add(InternalRow(-30L, str("baz"), 4129.8d)) + assert(!acc1.isZero) + checkResult(acc1.value, InternalRow(73L, str("baz"), 3L), expectedSchema, false) + + // Idempotency of result + checkResult(acc1.value, InternalRow(73L, str("baz"), 3L), expectedSchema, false) + + // A few updates to the copied accumulator + acc2.add(InternalRow(-2L, str("qwerty"), -6773.9d)) + acc2.add(InternalRow(-35L, str("zzz-top"), -323.9d)) + checkResult(acc2.value, InternalRow(-36L, str("zzz-top"), 2L), expectedSchema, false) + + // Merge accumulators + acc1.merge(acc2) + acc1.merge(acc2) + acc1.merge(accEmpty) + acc1.merge(accEmpty) + checkResult(acc1.value, InternalRow(1L, str("zzz-top"), 5L), expectedSchema, false) + + // Reset + acc1.reset() + assert(acc1.isZero) + } + + test("non-deterministic expressions") { + val acc_driver = AggregatingAccumulator( + Seq( + min(SparkPartitionID()) as "min_pid", + max(SparkPartitionID()) as "max_pid", + SparkPartitionID()), + Nil) + checkResult(acc_driver.value, InternalRow(null, null, 0), acc_driver.schema, false) + + def inPartition(id: Int)(f: => Unit): Unit = { + val ctx = new TaskContextImpl(0, 0, 1, 0, 0, null, new Properties, null) + TaskContext.setTaskContext(ctx) + try { + f + } finally { + TaskContext.unset() + } + } + + val acc1 = acc_driver.copy() + inPartition(3) { + acc1.add(InternalRow.empty) + } + val acc2 = acc_driver.copy() + inPartition(42) { + acc2.add(InternalRow.empty) + } + val acc3 = acc_driver.copy() + inPartition(96) { + acc3.add(InternalRow.empty) + } + + acc_driver.merge(acc1) + acc_driver.merge(acc2) + acc_driver.merge(acc3) + assert(!acc_driver.isZero) + checkResult(acc_driver.value, InternalRow(3, 96, 0), acc_driver.schema, false) + } + + test("collect agg metrics on job") { + val acc = AggregatingAccumulator( + Seq( + avg(a) + 1.0d as "avg_a", + sum(a + 10L) as "sum_a", + min(b) as "min_b", + max(b) as "max_b", + approxCountDistinct(b) as "acntd_b", + SortArray(CollectSet(If(a < 1000L, a % 3L, a % 6L)).toAggregateExpression(), true) + as "item_set", + min(SparkPartitionID()) as "min_pid", + max(SparkPartitionID()) as "max_pid", + SparkPartitionID()), + Seq(a, b)) + sparkContext.register(acc) + def consume(ids: Iterator[Long]): Unit = { + val row = new SpecificInternalRow(Seq(LongType, StringType)) + ids.foreach { id => + // Create the new row values. + row.setLong(0, id) + row.update(1, UTF8String.fromString(f"val_$id%06d")) + + // Update the accumulator + acc.add(row) + } + } + + // Run job 1 + spark.sparkContext + .range(0, 1000, 1, 8) + .foreachPartition(consume) + assert(checkResult( + acc.value, + InternalRow( + 500.5d, + 509500L, + str("val_000000"), + str("val_000999"), + 1057L, + new GenericArrayData(Seq(0L, 1L, 2L)), + 0, + 7, + 0), + acc.schema, + false)) + + // Run job 2 + spark.sparkContext + .range(1000, 1200, 1, 8) + .foreachPartition(consume) + assert(checkResult( + acc.value, + InternalRow( + 600.5d, + 731400L, + str("val_000000"), + str("val_001199"), + 1280L, + new GenericArrayData(Seq(0L, 1L, 2L, 3L, 4L, 5L)), + 0, + 7, + 0), + acc.schema, + false)) + } +} 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 index 76006efda992f..987338cf6cbbf 100644 --- 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 @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} +import org.apache.spark.sql.streaming.StreamTest -class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession { +class QueryPlanningTrackerEndToEndSuite extends StreamTest { + import testImplicits._ test("programmatic API") { val df = spark.range(1000).selectExpr("count(*)") @@ -38,4 +40,22 @@ class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession { assert(tracker.rules.nonEmpty) } + test("SPARK-29227: Track rule info in optimization phase in streaming") { + val inputData = MemoryStream[Int] + val df = inputData.toDF() + + def assertStatus(stream: StreamExecution): Unit = { + stream.processAllAvailable() + val tracker = stream.lastExecution.tracker + assert(tracker.phases.keys == Set("analysis", "optimization", "planning")) + assert(tracker.rules.nonEmpty) + } + + testStream(df)( + StartStream(), + AddData(inputData, 1, 2, 3), + Execute(assertStatus), + StopStream) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala index f6b006b98edd1..8bf7fe62cd49b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.util.Properties +import scala.collection.parallel.immutable.ParRange import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} @@ -46,7 +46,7 @@ class SQLExecutionSuite extends SparkFunSuite { import spark.implicits._ try { // Should not throw IllegalArgumentException - (1 to 100).par.foreach { _ => + new ParRange(1 to 100).foreach { _ => spark.sparkContext.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count() } } finally { 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 ff84b05713676..8ff293146127a 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 @@ -136,12 +136,14 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { assertNoSuchTable(s"ALTER TABLE $viewName SET SERDE 'whatever'") assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'") assertNoSuchTable(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')") - assertNoSuchTable(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'") assertNoSuchTable(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") assertNoSuchTable(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')") assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") assertNoSuchTable(s"ALTER TABLE $viewName RECOVER PARTITIONS") + + // For v2 ALTER TABLE statements, we have better error message saying view is not supported. + assertViewNotSupported(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") } } @@ -175,6 +177,11 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } + private def assertViewNotSupported(query: String): Unit = { + val e = intercept[AnalysisException](sql(query)) + assert(e.message.contains("'testView' is a view not a table")) + } + test("error handling: insert/load/truncate table commands against a view") { val viewName = "testView" withView(viewName) { 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 b751fb7c50438..0c766429742df 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,8 +22,7 @@ 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.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} -import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumnStatement, DescribeTableStatement, LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -217,68 +216,6 @@ class SparkSqlParserSuite extends AnalysisTest { assertEqual("DESCRIBE " + query, DescribeQueryCommand(query, parser.parsePlan(query))) } - test("analyze table statistics") { - assertEqual("analyze table t compute statistics", - AnalyzeTableCommand(TableIdentifier("t"), noscan = false)) - assertEqual("analyze table t compute statistics noscan", - AnalyzeTableCommand(TableIdentifier("t"), noscan = true)) - assertEqual("analyze table t partition (a) compute statistics nOscAn", - AnalyzePartitionCommand(TableIdentifier("t"), Map("a" -> None), noscan = true)) - - // Partitions specified - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> Some("2008-04-09")))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> None))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> None))) - assertEqual("ANALYZE TABLE t PARTITION(ds, hr=11) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> None, "hr" -> Some("11")))) - assertEqual("ANALYZE TABLE t PARTITION(ds, hr) COMPUTE STATISTICS", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, - partitionSpec = Map("ds" -> None, "hr" -> None))) - assertEqual("ANALYZE TABLE t PARTITION(ds, hr) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> None, "hr" -> None))) - - intercept("analyze table t compute statistics xxxx", - "Expected `NOSCAN` instead of `xxxx`") - intercept("analyze table t partition (a) compute statistics xxxx", - "Expected `NOSCAN` instead of `xxxx`") - } - - test("analyze table column statistics") { - intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS", "") - - assertEqual("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS key, value", - AnalyzeColumnCommand(TableIdentifier("t"), Option(Seq("key", "value")), allColumns = false)) - - // Partition specified - should be ignored - assertEqual("ANALYZE TABLE t PARTITION(ds='2017-06-10') " + - "COMPUTE STATISTICS FOR COLUMNS key, value", - AnalyzeColumnCommand(TableIdentifier("t"), Option(Seq("key", "value")), allColumns = false)) - - // Partition specified should be ignored in case of COMPUTE STATISTICS FOR ALL COLUMNS - assertEqual("ANALYZE TABLE t PARTITION(ds='2017-06-10') " + - "COMPUTE STATISTICS FOR ALL COLUMNS", - AnalyzeColumnCommand(TableIdentifier("t"), None, allColumns = true)) - - intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR ALL COLUMNS key, value", - "mismatched input 'key' expecting ") - intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR ALL", - "missing 'COLUMNS' at ''") - } - test("query organization") { // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows val baseSql = "select * from t" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index 392cce54ebede..ef81f1b788496 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -57,7 +57,7 @@ class UnsafeFixedWidthAggregationMapSuite private var taskContext: TaskContext = null - def testWithMemoryLeakDetection(name: String)(f: => Unit) { + def testWithMemoryLeakDetection(name: String)(f: => Unit): Unit = { def cleanup(): Unit = { if (taskMemoryManager != null) { assert(taskMemoryManager.cleanUpAllAllocatedMemory() === 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 index 37b106c3ea530..649467a27d93c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -19,7 +19,6 @@ 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, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf @@ -77,6 +76,13 @@ class AdaptiveQueryExecSuite } } + private def checkNumLocalShuffleReaders(plan: SparkPlan, expected: Int): Unit = { + val localReaders = collect(plan) { + case reader: LocalShuffleReaderExec => reader + } + assert(localReaders.length === expected) + } + test("Change merge join to broadcast join") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", @@ -87,6 +93,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -103,14 +110,7 @@ class AdaptiveQueryExecSuite 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) - } + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -125,6 +125,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -139,6 +140,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -160,6 +163,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) + // The child of remaining one BroadcastHashJoin is not ShuffleQueryStage. + // So only two LocalShuffleReader. + checkNumLocalShuffleReaders(adaptivePlan, 2) } } @@ -183,6 +189,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) + // The child of remaining two BroadcastHashJoin is not ShuffleQueryStage. + // So only two LocalShuffleReader. + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -206,6 +215,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) + // The child of remaining two BroadcastHashJoin is not ShuffleQueryStage. + // So only two LocalShuffleReader. + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -220,6 +232,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 2) + checkNumLocalShuffleReaders(adaptivePlan, 2) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.size == 1) } @@ -236,6 +250,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.size == 1) } @@ -254,6 +270,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) val sub = findReusedSubquery(adaptivePlan) @@ -273,6 +291,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) val sub = findReusedSubquery(adaptivePlan) @@ -295,6 +315,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) assert(ex.head.plan.isInstanceOf[BroadcastQueryStageExec]) @@ -355,6 +377,27 @@ class AdaptiveQueryExecSuite } } + test("Change merge join to broadcast join without local shuffle reader") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 2) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + // additional shuffle exchange introduced, so revert OptimizeLocalShuffleReader rule. + checkNumLocalShuffleReaders(adaptivePlan, 0) + } + } + test("Avoid changing merge join to broadcast join if too many empty partitions on build plan") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala index dc67446460877..3e47fd4289bef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala @@ -140,7 +140,7 @@ class SortBasedAggregationStoreSuite extends SparkFunSuite with LocalSparkConte } override def getKey(): UnsafeRow = key override def getValue(): UnsafeRow = value - override def close(): Unit = Unit + override def close(): Unit = () } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index df0f87e483cdc..9cfa9070aca67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.benchmark import java.sql.Timestamp import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -36,7 +37,12 @@ import org.apache.spark.sql.internal.SQLConf */ object DateTimeBenchmark extends SqlBasedBenchmark { private def doBenchmark(cardinality: Int, exprs: String*): Unit = { - spark.range(cardinality).selectExpr(exprs: _*).write.format("noop").save() + spark.range(cardinality) + .selectExpr(exprs: _*) + .write + .format("noop") + .mode(Overwrite) + .save() } private def run(cardinality: Int, name: String, exprs: String*): Unit = { @@ -132,7 +138,10 @@ object DateTimeBenchmark extends SqlBasedBenchmark { benchmark.addCase("From java.sql.Timestamp", numIters) { _ => spark.range(rowsNum) .map(millis => new Timestamp(millis)) - .write.format("noop").save() + .write + .format("noop") + .mode(Overwrite) + .save() } benchmark.addCase("Collect longs", numIters) { _ => spark.range(0, rowsNum, 1, 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index a109b11b2d6d3..941649df6b727 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.benchmark import java.time.Instant import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -44,6 +45,7 @@ object ExtractBenchmark extends SqlBasedBenchmark { .selectExpr(exprs: _*) .write .format("noop") + .mode(Overwrite) .save() } } @@ -59,8 +61,10 @@ object ExtractBenchmark extends SqlBasedBenchmark { } private def castExpr(from: String): String = from match { - case "timestamp" => s"cast(id as timestamp)" - case "date" => s"cast(cast(id as timestamp) as date)" + case "timestamp" => "cast(id as timestamp)" + case "date" => "cast(cast(id as timestamp) as date)" + case "interval" => "(cast(cast(id as timestamp) as date) - date'0001-01-01') + " + + "(cast(id as timestamp) - timestamp'1000-01-01 01:02:03.123456')" case other => throw new IllegalArgumentException( s"Unsupported column type $other. Valid column types are 'timestamp' and 'date'") } @@ -72,8 +76,8 @@ object ExtractBenchmark extends SqlBasedBenchmark { field: String, from: String): Unit = { val expr = func match { - case "extract" => s"EXTRACT($field FROM ${castExpr(from)})" - case "date_part" => s"DATE_PART('$field', ${castExpr(from)})" + case "extract" => s"EXTRACT($field FROM ${castExpr(from)}) AS $field" + case "date_part" => s"DATE_PART('$field', ${castExpr(from)}) AS $field" case other => throw new IllegalArgumentException( s"Unsupported function '$other'. Valid functions are 'extract' and 'date_part'.") } @@ -82,24 +86,36 @@ object ExtractBenchmark extends SqlBasedBenchmark { } } + private case class Settings(fields: Seq[String], func: Seq[String], iterNum: Long) + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val N = 10000000L - val fields = Seq( + val datetimeFields = Seq( "MILLENNIUM", "CENTURY", "DECADE", "YEAR", "ISOYEAR", "QUARTER", "MONTH", "WEEK", "DAY", "DAYOFWEEK", "DOW", "ISODOW", "DOY", "HOUR", "MINUTE", "SECOND", "MILLISECONDS", "MICROSECONDS", "EPOCH") + val intervalFields = Seq( + "MILLENNIUM", "CENTURY", "DECADE", "YEAR", + "QUARTER", "MONTH", "DAY", + "HOUR", "MINUTE", "SECOND", + "MILLISECONDS", "MICROSECONDS", "EPOCH") + val settings = Map( + "timestamp" -> Settings(datetimeFields, Seq("extract", "date_part"), N), + "date" -> Settings(datetimeFields, Seq("extract", "date_part"), N), + "interval" -> Settings(intervalFields, Seq("date_part"), N)) + + for { + (dataType, Settings(fields, funcs, iterNum)) <- settings + func <- funcs} { - Seq("extract", "date_part").foreach { func => - Seq("timestamp", "date").foreach { dateType => - val benchmark = new Benchmark(s"Invoke $func for $dateType", N, output = output) + val benchmark = new Benchmark(s"Invoke $func for $dataType", N, output = output) - run(benchmark, N, s"cast to $dateType", castExpr(dateType)) - fields.foreach(run(benchmark, func, N, _, dateType)) + run(benchmark, iterNum, s"cast to $dataType", castExpr(dataType)) + fields.foreach(run(benchmark, func, iterNum, _, dataType)) - benchmark.run() - } + benchmark.run() } } } 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 index 611f582b66605..6a6851ab60e5c 100644 --- 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 @@ -167,7 +167,7 @@ object InExpressionBenchmark extends SqlBasedBenchmark { def testClosure(): Unit = { val df = spark.sql(s"SELECT * FROM t WHERE id IN (${values.mkString(",")})") - df.queryExecution.toRdd.foreach(_ => Unit) + df.queryExecution.toRdd.foreach(_ => ()) } benchmark.addCase("In expression") { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala new file mode 100644 index 0000000000000..d75cb1040f31e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.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.execution.benchmark + +import scala.collection.mutable.ListBuffer + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.Column +import org.apache.spark.sql.SaveMode.Overwrite +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for interval 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/IntervalBenchmark-results.txt". + * }}} + */ +object IntervalBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + + private def doBenchmark(cardinality: Long, exprs: Column*): Unit = { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + spark + .range(0, cardinality, 1, 1) + .select(exprs: _*) + .write + .format("noop") + .mode(Overwrite) + .save() + } + } + + private def addCase( + benchmark: Benchmark, + cardinality: Long, + name: String, + exprs: Column*): Unit = { + benchmark.addCase(name, numIters = 3) { _ => + doBenchmark(cardinality, exprs: _*) + } + } + + private def buildString(withPrefix: Boolean, units: Seq[String] = Seq.empty): Column = { + val init = lit(if (withPrefix) "interval" else "") :: + ($"id" % 10000).cast("string") :: + lit("years") :: Nil + + concat_ws(" ", (init ++ units.map(lit)): _*) + } + + private def addCase(benchmark: Benchmark, cardinality: Long, units: Seq[String]): Unit = { + Seq(true, false).foreach { withPrefix => + val expr = buildString(withPrefix, units).cast("interval") + val note = if (withPrefix) "w/ interval" else "w/o interval" + benchmark.addCase(s"${units.length + 1} units $note", numIters = 3) { _ => + doBenchmark(cardinality, expr) + } + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val N = 1000000 + val timeUnits = Seq( + "13 months", "100 weeks", "9 days", "12 hours", + "5 minutes", "45 seconds", "123 milliseconds", "567 microseconds") + val intervalToTest = ListBuffer[String]() + + val benchmark = new Benchmark("cast strings to intervals", N, output = output) + // The first 2 cases are used to show the overhead of preparing the interval string. + addCase(benchmark, N, "prepare string w/ interval", buildString(true, timeUnits)) + addCase(benchmark, N, "prepare string w/o interval", buildString(false, timeUnits)) + addCase(benchmark, N, intervalToTest) // Only years + + for (unit <- timeUnits) { + intervalToTest.append(unit) + addCase(benchmark, N, intervalToTest) + } + + benchmark.run() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala index 7f7908544693f..9e5aca70ac628 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -41,6 +42,7 @@ object MakeDateTimeBenchmark extends SqlBasedBenchmark { .selectExpr(exprs: _*) .write .format("noop") + .mode(Overwrite) .save() } } 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 index 96f90f29707d2..4b6da5a02eac3 100644 --- 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -47,7 +48,11 @@ abstract class NestedSchemaPruningBenchmark extends SqlBasedBenchmark { private def addCase(benchmark: Benchmark, name: String, sql: String): Unit = { benchmark.addCase(name) { _ => - spark.sql(sql).write.format("noop").save() + spark.sql(sql) + .write + .format("noop") + .mode(Overwrite) + .save() } } 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 3760539c16841..e265e707b6b84 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 @@ -38,7 +38,8 @@ import org.apache.spark.util.random.XORShiftRandom */ object SortBenchmark extends BenchmarkBase { - private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { + private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, + refCmp: PrefixComparator): Unit = { val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort(buf, lo, hi, (r1: RecordPointerAndKeyPrefix, r2: RecordPointerAndKeyPrefix) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index 93006d05b75bc..5ff33b9cfbfc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -29,11 +29,19 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation /** * Benchmark to measure TPCDS query performance. * To run this: - * spark-submit --class --data-location + * {{{ + * 1. without sbt: + * bin/spark-submit --class --data-location + * 2. build/sbt "sql/test:runMain --data-location " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt + * "sql/test:runMain --data-location " + * Results will be written to "benchmarks/TPCDSQueryBenchmark-results.txt". + * }}} */ -object TPCDSQueryBenchmark extends Logging { - val conf = - new SparkConf() +object TPCDSQueryBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() .setMaster("local[1]") .setAppName("test-sql-context") .set("spark.sql.parquet.compression.codec", "snappy") @@ -43,7 +51,8 @@ object TPCDSQueryBenchmark extends Logging { .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) .set("spark.sql.crossJoin.enabled", "true") - val spark = SparkSession.builder.config(conf).getOrCreate() + SparkSession.builder.config(conf).getOrCreate() + } val tables = Seq("catalog_page", "catalog_returns", "customer", "customer_address", "customer_demographics", "date_dim", "household_demographics", "inventory", "item", @@ -80,13 +89,11 @@ object TPCDSQueryBenchmark extends Logging { case _ => } val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum - val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 5) + val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 2, output = output) benchmark.addCase(s"$name$nameSuffix") { _ => spark.sql(queryString).collect() } - logInfo(s"\n\n===== TPCDS QUERY BENCHMARK OUTPUT FOR $name =====\n") benchmark.run() - logInfo(s"\n\n===== FINISHED $name =====\n") } } @@ -100,8 +107,8 @@ object TPCDSQueryBenchmark extends Logging { } } - def main(args: Array[String]): Unit = { - val benchmarkArgs = new TPCDSQueryBenchmarkArguments(args) + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val benchmarkArgs = new TPCDSQueryBenchmarkArguments(mainArgs) // List of all TPC-DS v1.4 queries val tpcdsQueries = Seq( 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 index 9cbd6423f667f..04c1b5ade12cb 100644 --- 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ @@ -42,8 +43,12 @@ object UDFBenchmark extends SqlBasedBenchmark { 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() + spark.range(cardinality) + .select(udf(idCol, nullableIntCol, stringCol)) + .write + .format("noop") + .mode(Overwrite) + .save() } private def doRunBenchmarkWithPrimitiveTypes( @@ -51,7 +56,12 @@ object UDFBenchmark extends SqlBasedBenchmark { 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() + spark.range(cardinality) + .select(udf(idCol, nullableIntCol)) + .write + .format("noop") + .mode(Overwrite) + .save() } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { @@ -104,16 +114,25 @@ object UDFBenchmark extends SqlBasedBenchmark { 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() + spark.range(cardinality) + .select(col("id"), col("id") * 2, col("id") * 3) + .write + .format("noop") + .mode(Overwrite) + .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() + spark.range(cardinality) + .select( + identityUDF(col("id")), + identityUDF(col("id") * 2), + identityUDF(col("id") * 3)) + .write + .format("noop") + .mode(Overwrite) + .save() } benchmark.run() 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 0fac4dd3e5137..77047f329e105 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 @@ -38,7 +38,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { setupTestData() - private def cachePrimitiveTest(data: DataFrame, dataType: String) { + private def cachePrimitiveTest(data: DataFrame, dataType: String): Unit = { data.createOrReplaceTempView(s"testData$dataType") val storageLevel = MEMORY_ONLY val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala index 2d71a42628dfb..192db0e910d03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.BooleanType class BooleanBitSetSuite extends SparkFunSuite { import BooleanBitSet._ - def skeleton(count: Int) { + def skeleton(count: Int): Unit = { // ------------- // Tests encoder // ------------- @@ -87,7 +87,7 @@ class BooleanBitSetSuite extends SparkFunSuite { assert(!decoder.hasNext) } - def skeletonForDecompress(count: Int) { + def skeletonForDecompress(count: Int): Unit = { val builder = TestCompressibleColumnBuilder(new NoopColumnStats, BOOLEAN, BooleanBitSet) val rows = Seq.fill[InternalRow](count)(makeRandomRow(BOOLEAN)) val values = rows.map(_.getBoolean(0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index 28950b74cf1c8..61e4cc068fa80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -35,7 +35,7 @@ class DictionaryEncodingSuite extends SparkFunSuite { def testDictionaryEncoding[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T], - testDecompress: Boolean = true) { + testDecompress: Boolean = true): Unit = { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") @@ -49,7 +49,7 @@ class DictionaryEncodingSuite extends SparkFunSuite { seq.head +: seq.tail.filterNot(_ == seq.head) } - def skeleton(uniqueValueCount: Int, inputSeq: Seq[Int]) { + def skeleton(uniqueValueCount: Int, inputSeq: Seq[Int]): Unit = { // ------------- // Tests encoder // ------------- @@ -116,7 +116,7 @@ class DictionaryEncodingSuite extends SparkFunSuite { } } - def skeletonForDecompress(uniqueValueCount: Int, inputSeq: Seq[Int]) { + def skeletonForDecompress(uniqueValueCount: Int, inputSeq: Seq[Int]): Unit = { if (!testDecompress) return val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding) val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) 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 fb3388452e4e5..294feb72ea802 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 @@ -32,9 +32,9 @@ class IntegralDeltaSuite extends SparkFunSuite { def testIntegralDelta[I <: IntegralType]( columnStats: ColumnStats, columnType: NativeColumnType[I], - scheme: CompressionScheme) { + scheme: CompressionScheme): Unit = { - def skeleton(input: Seq[I#InternalType]) { + def skeleton(input: Seq[I#InternalType]): Unit = { // ------------- // Tests encoder // ------------- @@ -112,7 +112,7 @@ class IntegralDeltaSuite extends SparkFunSuite { assert(!decoder.hasNext) } - def skeletonForDecompress(input: Seq[I#InternalType]) { + def skeletonForDecompress(input: Seq[I#InternalType]): Unit = { val builder = TestCompressibleColumnBuilder(columnStats, columnType, scheme) val row = new GenericInternalRow(1) val nullRow = new GenericInternalRow(1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index b6f0b5e6277b4..01dac2d1ad74c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -35,11 +35,11 @@ class PassThroughSuite extends SparkFunSuite { def testPassThrough[T <: AtomicType]( columnStats: ColumnStats, - columnType: NativeColumnType[T]) { + columnType: NativeColumnType[T]): Unit = { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") - def skeleton(input: Seq[T#InternalType]) { + def skeleton(input: Seq[T#InternalType]): Unit = { // ------------- // Tests encoder // ------------- @@ -93,7 +93,7 @@ class PassThroughSuite extends SparkFunSuite { assert(!decoder.hasNext) } - def skeletonForDecompress(input: Seq[T#InternalType]) { + def skeletonForDecompress(input: Seq[T#InternalType]): Unit = { val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) val row = new GenericInternalRow(1) val nullRow = new GenericInternalRow(1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index eb1cdd9bbceff..29dbc13b59c6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -36,11 +36,11 @@ class RunLengthEncodingSuite extends SparkFunSuite { def testRunLengthEncoding[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T], - testDecompress: Boolean = true) { + testDecompress: Boolean = true): Unit = { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") - def skeleton(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]) { + def skeleton(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]): Unit = { // ------------- // Tests encoder // ------------- @@ -98,7 +98,7 @@ class RunLengthEncodingSuite extends SparkFunSuite { assert(!decoder.hasNext) } - def skeletonForDecompress(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]) { + def skeletonForDecompress(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]): Unit = { if (!testDecompress) return val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) 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 74ef81f7181da..a9b94bea9517a 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 @@ -74,46 +74,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { }.head } - test("create database") { - val sql = - """ - |CREATE DATABASE IF NOT EXISTS database_name - |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') - |COMMENT 'database_comment' LOCATION '/home/user/db' - """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = CreateDatabaseCommand( - "database_name", - ifNotExists = true, - Some("/home/user/db"), - Some("database_comment"), - Map("a" -> "a", "b" -> "b", "c" -> "c")) - 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')", - containsThesePhrases = Seq("key_without_value")) - } - test("drop database") { val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" @@ -184,6 +144,15 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { containsThesePhrases = Seq("key_without_value")) } + test("alter database set location") { + // ALTER (DATABASE|SCHEMA) database_name SET LOCATION + val sql1 = "ALTER DATABASE database_name SET LOCATION '/home/user/db'" + val parsed1 = parser.parsePlan(sql1) + + val expected1 = AlterDatabaseSetLocationCommand("database_name", "/home/user/db") + comparePlans(parsed1, expected1) + } + test("describe database") { // DESCRIBE DATABASE [EXTENDED] db_name; val sql1 = "DESCRIBE DATABASE EXTENDED db_name" @@ -802,17 +771,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { """.stripMargin) } - test("show databases") { - val sql1 = "SHOW DATABASES" - val sql2 = "SHOW DATABASES LIKE 'defau*'" - val parsed1 = parser.parsePlan(sql1) - val expected1 = ShowDatabasesCommand(None) - val parsed2 = parser.parsePlan(sql2) - val expected2 = ShowDatabasesCommand(Some("defau*")) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } - test("show tblproperties") { val parsed1 = parser.parsePlan("SHOW TBLPROPERTIES tab1") val expected1 = ShowTablePropertiesCommand(TableIdentifier("tab1", None), None) @@ -872,45 +830,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed4, expected4) } - - test("show partitions") { - val sql1 = "SHOW PARTITIONS t1" - val sql2 = "SHOW PARTITIONS db1.t1" - val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" - - val parsed1 = parser.parsePlan(sql1) - val expected1 = - ShowPartitionsCommand(TableIdentifier("t1", None), None) - val parsed2 = parser.parsePlan(sql2) - val expected2 = - ShowPartitionsCommand(TableIdentifier("t1", Some("db1")), None) - val expected3 = - ShowPartitionsCommand(TableIdentifier("t1", None), - Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))) - val parsed3 = parser.parsePlan(sql3) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - } - - test("support for other types in DBPROPERTIES") { - val sql = - """ - |CREATE DATABASE database_name - |LOCATION '/home/user/db' - |WITH DBPROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) - """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = CreateDatabaseCommand( - "database_name", - ifNotExists = false, - Some("/home/user/db"), - None, - Map("a" -> "1", "b" -> "0.1", "c" -> "true")) - - comparePlans(parsed, expected) - } - test("Test CTAS #1") { val s1 = """ @@ -1446,15 +1365,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") } - test("MSCK REPAIR table") { - val sql = "MSCK REPAIR TABLE tab1" - val parsed = parser.parsePlan(sql) - val expected = AlterTableRecoverPartitionsCommand( - TableIdentifier("tab1", None), - "MSCK REPAIR TABLE") - comparePlans(parsed, expected) - } - test("create table like") { val v1 = "CREATE TABLE table1 LIKE table2" val (target, source, location, exists) = parser.parsePlan(v1).collect { 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 fd1da2011f28e..8dad242a8d618 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 @@ -28,7 +28,7 @@ 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} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.internal.SQLConf @@ -150,9 +150,9 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { Seq(3 -> "c").toDF("i", "j").write.mode("append").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Nil) - Seq("c" -> 3).toDF("i", "j").write.mode("append").saveAsTable("t") + Seq(3.5 -> 3).toDF("i", "j").write.mode("append").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") - :: Row(null, "3") :: Nil) + :: Row(3, "3") :: Nil) Seq(4 -> "d").toDF("i", "j").write.saveAsTable("t1") @@ -757,6 +757,29 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Row("Description", "") :: Row("Location", CatalogUtils.URIToString(location)) :: Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) + + withTempDir { tmpDir => + if (isUsingHiveMetastore) { + val e1 = intercept[AnalysisException] { + sql(s"ALTER DATABASE $dbName SET LOCATION '${tmpDir.toURI}'") + } + assert(e1.getMessage.contains("does not support altering database location")) + } else { + sql(s"ALTER DATABASE $dbName SET LOCATION '${tmpDir.toURI}'") + val uriInCatalog = catalog.getDatabaseMetadata(dbNameWithoutBackTicks).locationUri + assert("file" === uriInCatalog.getScheme) + assert(new Path(tmpDir.getPath).toUri.getPath === uriInCatalog.getPath) + } + + intercept[NoSuchDatabaseException] { + sql(s"ALTER DATABASE `db-not-exist` SET LOCATION '${tmpDir.toURI}'") + } + + val e3 = intercept[IllegalArgumentException] { + sql(s"ALTER DATABASE $dbName SET LOCATION ''") + } + assert(e3.getMessage.contains("Can not create a Path from an empty string")) + } } finally { catalog.reset() } @@ -1134,7 +1157,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - protected def testRecoverPartitions() { + protected def testRecoverPartitions(): Unit = { val catalog = spark.sessionState.catalog // table to alter does not exist intercept[AnalysisException] { @@ -1372,7 +1395,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { // if (isUsingHiveMetastore) { // assert(storageFormat.properties.get("path") === expected) // } - assert(storageFormat.locationUri.map(_.getPath) === Some(expected.getPath)) + assert(storageFormat.locationUri === + Some(makeQualifiedPath(CatalogUtils.URIToString(expected)))) } // set table location sql("ALTER TABLE dbx.tab1 SET LOCATION '/path/to/your/lovely/heart'") @@ -1386,7 +1410,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { verifyLocation(new URI("/swanky/steak/place")) // set table partition location without explicitly specifying database sql("ALTER TABLE tab1 PARTITION (a='1', b='2') SET LOCATION 'vienna'") - verifyLocation(new URI("vienna"), Some(partSpec)) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("tab1")) + val viennaPartPath = new Path(new Path(table. location), "vienna") + verifyLocation(CatalogUtils.stringToURI(viennaPartPath.toString), Some(partSpec)) // table to alter does not exist intercept[AnalysisException] { sql("ALTER TABLE dbx.does_not_exist SET LOCATION '/mister/spark'") @@ -1550,13 +1576,11 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { "PARTITION (a='2', b='6') LOCATION 'paris' PARTITION (a='3', b='7')") assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2, part3)) assert(catalog.getPartition(tableIdent, part1).storage.locationUri.isDefined) - val partitionLocation = if (isUsingHiveMetastore) { - val tableLocation = catalog.getTableMetadata(tableIdent).storage.locationUri - assert(tableLocation.isDefined) - makeQualifiedPath(new Path(tableLocation.get.toString, "paris").toString) - } else { - new URI("paris") - } + + val tableLocation = catalog.getTableMetadata(tableIdent).storage.locationUri + assert(tableLocation.isDefined) + val partitionLocation = makeQualifiedPath( + new Path(tableLocation.get.toString, "paris").toString) assert(catalog.getPartition(tableIdent, part2).storage.locationUri == Option(partitionLocation)) assert(catalog.getPartition(tableIdent, part3).storage.locationUri.isDefined) @@ -2040,7 +2064,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { test("show functions") { withUserDefinedFunction("add_one" -> true) { - val numFunctions = FunctionRegistry.functionSet.size.toLong + val numFunctions = FunctionRegistry.functionSet.size.toLong + + FunctionsCommand.virtualOperators.size.toLong assert(sql("show functions").count() === numFunctions) assert(sql("show system functions").count() === numFunctions) assert(sql("show all functions").count() === numFunctions) @@ -2138,7 +2163,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { spark.sessionState.catalog.refreshTable(TableIdentifier("t")) val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) - assert(table1.location == newDir) + assert(table1.location == makeQualifiedPath(newDir.toString)) assert(!newDirFile.exists) spark.sql("INSERT INTO TABLE t SELECT 'c', 1") @@ -2501,6 +2526,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { assert(table.location.toString.startsWith("file:/")) } + withTempDir { dir => + assert(!dir.getAbsolutePath.startsWith("file:/")) + spark.sql(s"ALTER TABLE t SET LOCATION '$dir'") + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location.toString.startsWith("file:/")) + } + withTempDir { dir => assert(!dir.getAbsolutePath.startsWith("file:/")) // The parser does not recognize the backslashes on Windows as they are. @@ -2519,6 +2551,37 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("the qualified path of a partition is stored in the catalog") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a STRING, b STRING) + |USING ${dataSource} PARTITIONED BY(b) LOCATION '$dir' + """.stripMargin) + spark.sql("INSERT INTO TABLE t PARTITION(b=1) SELECT 2") + val part = spark.sessionState.catalog.getPartition(TableIdentifier("t"), Map("b" -> "1")) + assert(part.storage.locationUri.contains( + makeQualifiedPath(new File(dir, "b=1").getAbsolutePath))) + assert(part.storage.locationUri.get.toString.startsWith("file:/")) + } + withTempDir { dir => + spark.sql(s"ALTER TABLE t PARTITION(b=1) SET LOCATION '$dir'") + + val part = spark.sessionState.catalog.getPartition(TableIdentifier("t"), Map("b" -> "1")) + assert(part.storage.locationUri.contains(makeQualifiedPath(dir.getAbsolutePath))) + assert(part.storage.locationUri.get.toString.startsWith("file:/")) + } + + withTempDir { dir => + spark.sql(s"ALTER TABLE t ADD PARTITION(b=2) LOCATION '$dir'") + val part = spark.sessionState.catalog.getPartition(TableIdentifier("t"), Map("b" -> "2")) + assert(part.storage.locationUri.contains(makeQualifiedPath(dir.getAbsolutePath))) + assert(part.storage.locationUri.get.toString.startsWith("file:/")) + } + } + } + protected def testAddColumn(provider: String): Unit = { withTable("t1") { sql(s"CREATE TABLE t1 (c1 int) USING $provider") @@ -2599,7 +2662,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") } - assert(e.message.contains("ALTER ADD COLUMNS does not support views")) + assert(e.message.contains("'tmp_v' is a view not a table")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 7110f13e2ead9..562e61390a53d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -25,35 +25,67 @@ import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} -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.{AliasIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions.{EqualTo, IntegerLiteral, StringLiteral} 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.connector.{InMemoryTableCatalog, InMemoryTableProvider} -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog} -import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceResolution} -import org.apache.spark.sql.internal.SQLConf.DEFAULT_V2_CATALOG +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, SubqueryAlias, UpdateTable} +import org.apache.spark.sql.connector.InMemoryTableProvider +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.execution.datasources.CreateTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap class PlanResolutionSuite extends AnalysisTest { import CatalystSqlParser._ private val v2Format = classOf[InMemoryTableProvider].getName + private val table: Table = { + val t = mock(classOf[Table]) + when(t.schema()).thenReturn(new StructType().add("i", "int")) + t + } + private val testCat: TableCatalog = { - val newCatalog = new InMemoryTableCatalog - newCatalog.initialize("testcat", CaseInsensitiveStringMap.empty()) + val newCatalog = mock(classOf[TableCatalog]) + when(newCatalog.loadTable(any())).thenAnswer((invocation: InvocationOnMock) => { + invocation.getArgument[Identifier](0).name match { + case "tab" => + table + case name => + throw new NoSuchTableException(name) + } + }) + when(newCatalog.name()).thenReturn("testcat") newCatalog } - private val v2SessionCatalog = { - val newCatalog = new InMemoryTableCatalog - newCatalog.initialize("session", CaseInsensitiveStringMap.empty()) + private val v2SessionCatalog: TableCatalog = { + val newCatalog = mock(classOf[TableCatalog]) + when(newCatalog.loadTable(any())).thenAnswer((invocation: InvocationOnMock) => { + invocation.getArgument[Identifier](0).name match { + case "v1Table" => + val v1Table = mock(classOf[V1Table]) + when(v1Table.schema).thenReturn(new StructType().add("i", "int")) + v1Table + case "v2Table" => + table + case name => + throw new NoSuchTableException(name) + } + }) + when(newCatalog.name()).thenReturn(CatalogManager.SESSION_CATALOG_NAME) newCatalog } + private val v1SessionCatalog: SessionCatalog = new SessionCatalog( + new InMemoryCatalog, + EmptyFunctionRegistry, + new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) + private val catalogManagerWithDefault = { val manager = mock(classOf[CatalogManager]) when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => { @@ -64,8 +96,8 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) - when(manager.defaultCatalog).thenReturn(Some(testCat)) - when(manager.v2SessionCatalog).thenReturn(v2SessionCatalog) + when(manager.currentCatalog).thenReturn(testCat) + when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } @@ -79,20 +111,25 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) - when(manager.defaultCatalog).thenReturn(None) - when(manager.v2SessionCatalog).thenReturn(v2SessionCatalog) + when(manager.currentCatalog).thenReturn(v2SessionCatalog) + when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } def parseAndResolve(query: String, withDefault: Boolean = false): LogicalPlan = { - val newConf = conf.copy() - newConf.setConfString(DEFAULT_V2_CATALOG.key, "testcat") val catalogManager = if (withDefault) { catalogManagerWithDefault } else { catalogManagerWithoutDefault } - DataSourceResolution(newConf, catalogManager).apply(parsePlan(query)) + val analyzer = new Analyzer(catalogManager, conf) + val rules = Seq( + new ResolveCatalogs(catalogManager), + new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), + analyzer.ResolveTables) + rules.foldLeft(parsePlan(query)) { + case (plan, rule) => rule.apply(plan) + } } private def parseResolveCompare(query: String, expected: LogicalPlan): Unit = @@ -438,7 +475,7 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case create: CreateV2Table => - assert(create.catalog.name == "session") + assert(create.catalog.name == CatalogManager.SESSION_CATALOG_NAME) assert(create.tableName == Identifier.of(Array("mydb"), "page_view")) assert(create.tableSchema == new StructType() .add("id", LongType) @@ -542,7 +579,7 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case ctas: CreateTableAsSelect => - assert(ctas.catalog.name == "session") + assert(ctas.catalog.name == CatalogManager.SESSION_CATALOG_NAME) assert(ctas.tableName == Identifier.of(Array("mydb"), "page_view")) assert(ctas.properties == expectedProperties) assert(ctas.writeOptions.isEmpty) @@ -642,51 +679,223 @@ class PlanResolutionSuite extends AnalysisTest { // 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) + Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { + case (tblName, useV1Command) => + val sql1 = s"ALTER TABLE $tblName SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2 = s"ALTER TABLE $tblName UNSET TBLPROPERTIES ('comment', 'test')" + val sql3 = s"ALTER TABLE $tblName UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + val parsed1 = parseAndResolve(sql1) + val parsed2 = parseAndResolve(sql2) + val parsed3 = parseAndResolve(sql3) + + val tableIdent = TableIdentifier(tblName, None) + if (useV1Command) { + val expected1 = AlterTableSetPropertiesCommand( + tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) + val expected2 = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = false, isView = false) + val expected3 = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = true, isView = false) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } else { + parsed1 match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( + TableChange.setProperty("test", "test"), + TableChange.setProperty("comment", "new_comment"))) + case _ => fail("expect AlterTable") + } + + parsed2 match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( + TableChange.removeProperty("comment"), + TableChange.removeProperty("test"))) + case _ => fail("expect AlterTable") + } + + parsed3 match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( + TableChange.removeProperty("comment"), + TableChange.removeProperty("test"))) + case _ => fail("expect AlterTable") + } + } + } - 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 sql4 = "ALTER TABLE non_exist SET TBLPROPERTIES ('test' = 'test')" + val sql5 = "ALTER TABLE non_exist UNSET TBLPROPERTIES ('test')" + val parsed4 = parseAndResolve(sql4) + val parsed5 = parseAndResolve(sql5) - comparePlans(parsed1_table, expected1_table) - comparePlans(parsed2_table, expected2_table) - comparePlans(parsed3_table, expected3_table) + // For non-existing tables, we convert it to v2 command with `UnresolvedV2Table` + parsed4 match { + case AlterTable(_, _, _: UnresolvedV2Relation, _) => // OK + case _ => fail("Expect AlterTable, but got:\n" + parsed4.treeString) + } + parsed5 match { + case AlterTable(_, _, _: UnresolvedV2Relation, _) => // OK + case _ => fail("Expect AlterTable, but got:\n" + parsed5.treeString) + } } 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) + Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { + case (tblName, useV1Command) => + val sql = + s""" + |ALTER TABLE $tblName + |SET TBLPROPERTIES ('a' = 1, 'b' = 0.1, 'c' = TRUE) + """.stripMargin + val parsed = parseAndResolve(sql) + if (useV1Command) { + val expected = AlterTableSetPropertiesCommand( + TableIdentifier(tblName), + Map("a" -> "1", "b" -> "0.1", "c" -> "true"), + isView = false) + + comparePlans(parsed, expected) + } else { + parsed match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( + TableChange.setProperty("a", "1"), + TableChange.setProperty("b", "0.1"), + TableChange.setProperty("c", "true"))) + case _ => fail("Expect AlterTable, but got:\n" + parsed.treeString) + } + } + } } 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) + Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { + case (tblName, useV1Command) => + val sql = s"ALTER TABLE $tblName SET LOCATION 'new location'" + val parsed = parseAndResolve(sql) + if (useV1Command) { + val expected = AlterTableSetLocationCommand( + TableIdentifier(tblName, None), + None, + "new location") + comparePlans(parsed, expected) + } else { + parsed match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq(TableChange.setProperty("location", "new location"))) + case _ => fail("Expect AlterTable, but got:\n" + parsed.treeString) + } + } + } + } + + test("DESCRIBE TABLE") { + Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { + case (tblName, useV1Command) => + val sql1 = s"DESC TABLE $tblName" + val sql2 = s"DESC TABLE EXTENDED $tblName" + val parsed1 = parseAndResolve(sql1) + val parsed2 = parseAndResolve(sql2) + if (useV1Command) { + val expected1 = DescribeTableCommand(TableIdentifier(tblName, None), Map.empty, false) + val expected2 = DescribeTableCommand(TableIdentifier(tblName, None), Map.empty, true) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } else { + parsed1 match { + case DescribeTable(_: DataSourceV2Relation, isExtended) => + assert(!isExtended) + case _ => fail("Expect DescribeTable, but got:\n" + parsed1.treeString) + } + + parsed2 match { + case DescribeTable(_: DataSourceV2Relation, isExtended) => + assert(isExtended) + case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) + } + } + + val sql3 = s"DESC TABLE $tblName PARTITION(a=1)" + if (useV1Command) { + val parsed3 = parseAndResolve(sql3) + val expected3 = DescribeTableCommand( + TableIdentifier(tblName, None), Map("a" -> "1"), false) + comparePlans(parsed3, expected3) + } else { + val e = intercept[AnalysisException](parseAndResolve(sql3)) + assert(e.message.contains("DESCRIBE TABLE does not support partition for v2 tables")) + } + } + + // use v1 command to describe views. + val sql4 = "DESC TABLE v" + val parsed4 = parseAndResolve(sql4) + assert(parsed4.isInstanceOf[DescribeTableCommand]) + } + + test("UPDATE TABLE") { + Seq("v1Table", "v2Table", "testcat.tab").foreach { tblName => + val sql1 = s"UPDATE $tblName SET name='Robert', age=32" + val sql2 = s"UPDATE $tblName AS t SET name='Robert', age=32" + val sql3 = s"UPDATE $tblName AS t SET name='Robert', age=32 WHERE p=1" + + val parsed1 = parseAndResolve(sql1) + val parsed2 = parseAndResolve(sql2) + val parsed3 = parseAndResolve(sql3) + + parsed1 match { + case u @ UpdateTable( + _: DataSourceV2Relation, + Seq(name: UnresolvedAttribute, age: UnresolvedAttribute), + Seq(StringLiteral("Robert"), IntegerLiteral(32)), + None) => + assert(name.name == "name") + assert(age.name == "age") + + case _ => fail("Expect UpdateTable, but got:\n" + parsed1.treeString) + } + + parsed2 match { + case UpdateTable( + SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Seq(name: UnresolvedAttribute, age: UnresolvedAttribute), + Seq(StringLiteral("Robert"), IntegerLiteral(32)), + None) => + assert(name.name == "name") + assert(age.name == "age") + + case _ => fail("Expect UpdateTable, but got:\n" + parsed2.treeString) + } + + parsed3 match { + case UpdateTable( + SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Seq(name: UnresolvedAttribute, age: UnresolvedAttribute), + Seq(StringLiteral("Robert"), IntegerLiteral(32)), + Some(EqualTo(p: UnresolvedAttribute, IntegerLiteral(1)))) => + assert(name.name == "name") + assert(age.name == "age") + assert(p.name == "p") + + case _ => fail("Expect UpdateTable, but got:\n" + parsed3.treeString) + } + } + + val sql = "UPDATE non_existing SET id=1" + val parsed = parseAndResolve(sql) + parsed match { + case u: UpdateTable => + assert(u.table.isInstanceOf[UnresolvedV2Relation]) + case _ => fail("Expect UpdateTable, but got:\n" + parsed.treeString) + } } + + // TODO: add tests for more commands. } 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 4b086e830e456..a7a2349a1dfb9 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 @@ -416,6 +416,35 @@ class FileIndexSuite extends SharedSparkSession { } } + test("Add an option to ignore block locations when listing file") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=foo") + partitionDirectory.mkdir() + for (i <- 1 to 8) { + val file = new File(partitionDirectory, i + ".txt") + stringToFile(file, "text") + } + val path = new Path(dir.getCanonicalPath) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) + withSQLConf(SQLConf.IGNORE_DATA_LOCALITY.key -> "false", + "fs.file.impl" -> classOf[SpecialBlockLocationFileSystem].getName) { + val withBlockLocations = fileIndex. + listLeafFiles(Seq(new Path(partitionDirectory.getPath))) + + withSQLConf(SQLConf.IGNORE_DATA_LOCALITY.key -> "true") { + val withoutBlockLocations = fileIndex. + listLeafFiles(Seq(new Path(partitionDirectory.getPath))) + + assert(withBlockLocations.size == withoutBlockLocations.size) + assert(withBlockLocations.forall(b => b.isInstanceOf[LocatedFileStatus] && + b.asInstanceOf[LocatedFileStatus].getBlockLocations.nonEmpty)) + assert(withoutBlockLocations.forall(b => b.isInstanceOf[FileStatus] && + !b.isInstanceOf[LocatedFileStatus])) + assert(withoutBlockLocations.forall(withBlockLocations.contains)) + } + } + } + } } object DeletionRaceFileSystem { 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 d5502ba5737c0..5256043289d5e 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 @@ -99,13 +99,13 @@ class OrcReadSchemaSuite override val format: String = "orc" - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() originalConf = spark.conf.get(SQLConf.ORC_VECTORIZED_READER_ENABLED) spark.conf.set(SQLConf.ORC_VECTORIZED_READER_ENABLED.key, "false") } - override def afterAll() { + override def afterAll(): Unit = { spark.conf.set(SQLConf.ORC_VECTORIZED_READER_ENABLED.key, originalConf) super.afterAll() } @@ -124,13 +124,13 @@ class VectorizedOrcReadSchemaSuite override val format: String = "orc" - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() originalConf = spark.conf.get(SQLConf.ORC_VECTORIZED_READER_ENABLED) spark.conf.set(SQLConf.ORC_VECTORIZED_READER_ENABLED.key, "true") } - override def afterAll() { + override def afterAll(): Unit = { spark.conf.set(SQLConf.ORC_VECTORIZED_READER_ENABLED.key, originalConf) super.afterAll() } @@ -165,13 +165,13 @@ class ParquetReadSchemaSuite override val format: String = "parquet" - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() originalConf = spark.conf.get(SQLConf.PARQUET_VECTORIZED_READER_ENABLED) spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "false") } - override def afterAll() { + override def afterAll(): Unit = { spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, originalConf) super.afterAll() } @@ -187,13 +187,13 @@ class VectorizedParquetReadSchemaSuite override val format: String = "parquet" - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() originalConf = spark.conf.get(SQLConf.PARQUET_VECTORIZED_READER_ENABLED) spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") } - override def afterAll() { + override def afterAll(): Unit = { spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, originalConf) super.afterAll() } @@ -209,13 +209,13 @@ class MergedParquetReadSchemaSuite override val format: String = "parquet" - override def beforeAll() { + override def beforeAll(): Unit = { super.beforeAll() originalConf = spark.conf.get(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED) spark.conf.set(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key, "true") } - override def afterAll() { + override def afterAll(): Unit = { spark.conf.set(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key, originalConf) super.afterAll() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index bb3cec579016e..8cb5c12d76c97 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -269,7 +269,7 @@ abstract class SchemaPruningSuite checkAnswer(query, Row("Y.", 1) :: Row("X.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } - protected def testSchemaPruning(testName: String)(testThunk: => Unit) { + protected def testSchemaPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(vectorizedReaderEnabledKey -> "true") { withContacts(testThunk) @@ -293,7 +293,7 @@ abstract class SchemaPruningSuite } } - private def withContacts(testThunk: => Unit) { + private def withContacts(testThunk: => Unit): Unit = { withTempPath { dir => val path = dir.getCanonicalPath @@ -315,7 +315,7 @@ abstract class SchemaPruningSuite } } - private def withContactsWithDataPartitionColumn(testThunk: => Unit) { + private def withContactsWithDataPartitionColumn(testThunk: => Unit): Unit = { withTempPath { dir => val path = dir.getCanonicalPath @@ -381,7 +381,7 @@ abstract class SchemaPruningSuite // 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) { + private def testExactCaseQueryPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Case-sensitive parser - mixed-case schema - $testName") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { withMixedCaseData(testThunk) @@ -392,7 +392,7 @@ abstract class SchemaPruningSuite // 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) { + private def testMixedCaseQueryPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Case-insensitive parser - mixed-case schema - $testName") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { withMixedCaseData(testThunk) @@ -401,7 +401,7 @@ abstract class SchemaPruningSuite } // Tests given test function with Spark vectorized reader and non-vectorized reader. - private def withMixedCaseData(testThunk: => Unit) { + private def withMixedCaseData(testThunk: => Unit): Unit = { withDataSourceTable(mixedCaseData, "mixedcase") { testThunk } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index e41e81af508f1..a4cffedaf82d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -21,6 +21,7 @@ import java.time.{Instant, LocalDate} import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Column, Dataset, Row} +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -42,7 +43,9 @@ import org.apache.spark.sql.types._ object CSVBenchmark extends SqlBasedBenchmark { import spark.implicits._ - private def toNoop(ds: Dataset[_]): Unit = ds.write.format("noop").save() + private def toNoop(ds: Dataset[_]): Unit = { + ds.write.format("noop").mode(Overwrite).save() + } private def quotedValuesBenchmark(rowsNum: Int, numIters: Int): Unit = { val benchmark = new Benchmark(s"Parsing quoted values", rowsNum, output = output) 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 835fa4ef166b2..48a8c92aad759 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 @@ -50,6 +50,8 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { private val carsFile8859 = "test-data/cars_iso-8859-1.csv" private val carsTsvFile = "test-data/cars.tsv" private val carsAltFile = "test-data/cars-alternative.csv" + private val carsMultiCharDelimitedFile = "test-data/cars-multichar-delim.csv" + private val carsMultiCharCrazyDelimitedFile = "test-data/cars-multichar-delim-crazy.csv" private val carsUnbalancedQuotesFile = "test-data/cars-unbalanced-quotes.csv" private val carsNullFile = "test-data/cars-null.csv" private val carsEmptyValueFile = "test-data/cars-empty-value.csv" @@ -188,6 +190,49 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { verifyCars(cars, withHeader = true) } + test("test with tab delimiter and double quote") { + val cars = spark.read + .options(Map("quote" -> "\"", "delimiter" -> """\t""", "header" -> "true")) + .csv(testFile(carsTsvFile)) + + verifyCars(cars, numFields = 6, withHeader = true, checkHeader = false) + } + + test("SPARK-24540: test with multiple character delimiter (comma space)") { + val cars = spark.read + .options(Map("quote" -> "\'", "delimiter" -> ", ", "header" -> "true")) + .csv(testFile(carsMultiCharDelimitedFile)) + + verifyCars(cars, withHeader = true) + } + + test("SPARK-24540: test with multiple (crazy) character delimiter") { + val cars = spark.read + .options(Map("quote" -> "\'", "delimiter" -> """_/-\\_""", "header" -> "true")) + .csv(testFile(carsMultiCharCrazyDelimitedFile)) + + verifyCars(cars, withHeader = true) + + // check all the other columns, besides year (which is covered by verifyCars) + val otherCols = cars.select("make", "model", "comment", "blank").collect() + val expectedOtherColVals = Seq( + ("Tesla", "S", "No comment", null), + ("Ford", "E350", "Go get one now they are going fast", null), + ("Chevy", "Volt", null, null) + ) + + expectedOtherColVals.zipWithIndex.foreach { case (values, index) => + val actualRow = otherCols(index) + values match { + case (make, model, comment, blank) => + assert(make == actualRow.getString(0)) + assert(model == actualRow.getString(1)) + assert(comment == actualRow.getString(2)) + assert(blank == actualRow.getString(3)) + } + } + } + test("parse unescaped quotes with maxCharsPerColumn") { val rows = spark.read .format("csv") @@ -820,8 +865,8 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { .load(testFile(simpleSparseFile)) assert( - df.schema.fields.map(field => field.dataType).deep == - Array(IntegerType, IntegerType, IntegerType, IntegerType).deep) + df.schema.fields.map(field => field.dataType).sameElements( + Array(IntegerType, IntegerType, IntegerType, IntegerType))) } test("old csv data source name works") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala index f486e603e2552..58e710a7d66b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -21,6 +21,7 @@ import java.time.{Instant, LocalDate} import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -39,7 +40,7 @@ import org.apache.spark.sql.types._ * }}} */ -object JSONBenchmark extends SqlBasedBenchmark { +object JsonBenchmark extends SqlBasedBenchmark { import spark.implicits._ private def prepareDataInfo(benchmark: Benchmark): Unit = { @@ -49,7 +50,7 @@ object JSONBenchmark extends SqlBasedBenchmark { } private def run(ds: Dataset[_]): Unit = { - ds.write.format("noop").save() + ds.write.format("noop").mode(Overwrite).save() } def schemaInferring(rowsNum: Int, numIters: Int): Unit = { 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 2998e673bd45c..3574aa266b35f 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 @@ -49,7 +49,7 @@ class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { import testImplicits._ test("Type promotion") { - def checkTypePromotion(expected: Any, actual: Any) { + def checkTypePromotion(expected: Any, actual: Any): Unit = { assert(expected.getClass == actual.getClass, s"Failed to promote ${actual.getClass} to ${expected.getClass}.") assert(expected == actual, @@ -128,7 +128,7 @@ class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { } test("Get compatible type") { - def checkDataType(t1: DataType, t2: DataType, expected: DataType) { + def checkDataType(t1: DataType, t2: DataType, expected: DataType): Unit = { var actual = JsonInferSchema.compatibleType(t1, t2) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") 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 index c5a03cb8ef6d3..b4073bedf5597 100644 --- 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 @@ -32,6 +32,7 @@ class NoopSuite extends SharedSparkSession { } .write .format("noop") + .mode("append") .save() assert(accum.value == numElems) } @@ -54,7 +55,7 @@ class NoopSuite extends SharedSparkSession { accum.add(1) x } - .write.format("noop").save() + .write.mode("append").format("noop").save() assert(accum.value == numElems) } } 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 a2d96dd0468b4..1e27593584786 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 @@ -60,7 +60,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { .createOrReplaceTempView("orc_temp_table") } - protected def testBloomFilterCreation(bloomFilterKind: Kind) { + protected def testBloomFilterCreation(bloomFilterKind: Kind): Unit = { val tableName = "bloomFilter" withTempDir { dir => @@ -120,7 +120,8 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } - protected def testSelectiveDictionaryEncoding(isSelective: Boolean, isHive23: Boolean = false) { + protected def testSelectiveDictionaryEncoding(isSelective: Boolean, + isHive23: Boolean = false): Unit = { val tableName = "orcTable" withTempDir { dir => @@ -345,7 +346,9 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } - test("SPARK-23340 Empty float/double array columns raise EOFException") { + // SPARK-28885 String value is not allowed to be stored as numeric type with + // ANSI store assignment policy. + ignore("SPARK-23340 Empty float/double array columns raise EOFException") { Seq(Seq(Array.empty[Float]).toDF(), Seq(Array.empty[Double]).toDF()).foreach { df => withTempPath { path => df.write.format("orc").save(path.getCanonicalPath) 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 026ba5deffdfd..39590b063f0af 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 @@ -66,7 +66,7 @@ private[parquet] class TestGroupWriteSupport(schema: MessageType) extends WriteS new WriteContext(schema, new java.util.HashMap[String, String]()) } - override def write(record: Group) { + override def write(record: Group): Unit = { groupWriter.write(record) } } 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 88b94281d88ee..f38973f7dffd1 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 @@ -162,9 +162,9 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS test("SPARK-10634 timestamp written and read as INT64 - truncation") { withTable("ts") { sql("create table ts (c1 int, c2 timestamp) using parquet") - sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") + sql("insert into ts values (1, timestamp'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')") + sql("insert into ts values (3, timestamp'1965-01-01 10:11:12.123456')") val expected = Seq( (1, "2016-01-01 10:11:12.123456"), (2, null), @@ -177,13 +177,13 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS withTable("ts") { withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") { sql("create table ts (c1 int, c2 timestamp) using parquet") - sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") + sql("insert into ts values (1, timestamp'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.125456')") - sql("insert into ts values (4, '1965-01-01 10:11:12.125')") - 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')") + sql("insert into ts values (3, timestamp'1965-01-01 10:11:12.125456')") + sql("insert into ts values (4, timestamp'1965-01-01 10:11:12.125')") + sql("insert into ts values (5, timestamp'1965-01-01 10:11:12.1')") + sql("insert into ts values (6, timestamp'1965-01-01 10:11:12.123456789')") + sql("insert into ts values (7, timestamp'0001-01-01 00:00:00.000000')") val expected = Seq( (1, "2016-01-01 10:11:12.123"), (2, null), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala index d02014c0dee54..61c9782bd175d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala @@ -21,8 +21,8 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.Mockito.when import org.scalatest.concurrent.Eventually -import org.scalatest.mockito.MockitoSugar import org.scalatest.time.SpanSugar._ +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index 1ec9986328429..06077c94b66fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -33,7 +33,7 @@ class RowQueueSuite extends SparkFunSuite with EncryptionFunSuite { test("in-memory queue") { val page = MemoryBlock.fromLongArray(new Array[Long](1<<10)) val queue = new InMemoryRowQueue(page, 1) { - override def close() {} + override def close(): Unit = {} } val row = new UnsafeRow(1) row.pointTo(new Array[Byte](16), 16) 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 b1e6459838151..f791ab66e86fa 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 @@ -42,7 +42,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class TextSocketStreamSuite extends StreamTest with SharedSparkSession { - override def afterEach() { + override def afterEach(): Unit = { sqlContext.streams.active.foreach(_.stop()) if (serverThread != null) { serverThread.interrupt() 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 a84d107f2cbc0..488879938339d 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 @@ -406,7 +406,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] var latestStoreVersion = 0 - def generateStoreVersions() { + def generateStoreVersions(): Unit = { for (i <- 1 to 20) { val store = StateStore.get(storeProviderId, keySchema, valueSchema, None, latestStoreVersion, storeConf, hadoopConf) @@ -586,7 +586,8 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] query.processAllAvailable() require(query.lastProgress != null) // at least one batch processed after start val loadedProvidersMethod = - PrivateMethod[mutable.HashMap[StateStoreProviderId, StateStoreProvider]]('loadedProviders) + PrivateMethod[mutable.HashMap[StateStoreProviderId, StateStoreProvider]]( + Symbol("loadedProviders")) val loadedProvidersMap = StateStore invokePrivate loadedProvidersMethod() val loadedProviders = loadedProvidersMap.synchronized { loadedProvidersMap.values.toSeq } query.stop() @@ -781,7 +782,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] provider: HDFSBackedStateStoreProvider, version: Long, isSnapshot: Boolean): Boolean = { - val method = PrivateMethod[Path]('baseDir) + val method = PrivateMethod[Path](Symbol("baseDir")) val basePath = provider invokePrivate method() val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" val filePath = new File(basePath.toString, fileName) @@ -789,7 +790,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } def deleteFilesEarlierThanVersion(provider: HDFSBackedStateStoreProvider, version: Long): Unit = { - val method = PrivateMethod[Path]('baseDir) + val method = PrivateMethod[Path](Symbol("baseDir")) val basePath = provider invokePrivate method() for (version <- 0 until version.toInt) { for (isSnapshot <- Seq(false, true)) { @@ -804,7 +805,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] provider: HDFSBackedStateStoreProvider, version: Long, isSnapshot: Boolean): Unit = { - val method = PrivateMethod[Path]('baseDir) + val method = PrivateMethod[Path](Symbol("baseDir")) val basePath = provider invokePrivate method() val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" val filePath = new File(basePath.toString, fileName) 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 index 9e42056c19a0c..298afa880c930 100644 --- 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 @@ -73,7 +73,7 @@ class AllExecutionsPageSuite extends SharedSparkSession with BeforeAndAfter { 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("illegalargumentexception")) assert(html.contains("duration")) } 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 758780c80b284..c0f4bb4372bbf 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 @@ -1067,7 +1067,8 @@ class ColumnarBatchSuite extends SparkFunSuite { } } - private def compareStruct(fields: Seq[StructField], r1: InternalRow, r2: Row, seed: Long) { + private def compareStruct(fields: Seq[StructField], r1: InternalRow, r2: Row, + seed: Long): Unit = { fields.zipWithIndex.foreach { case (field: StructField, ordinal: Int) => assert(r1.isNullAt(ordinal) == r2.isNullAt(ordinal), "Seed = " + seed) if (!r1.isNullAt(ordinal)) { @@ -1159,7 +1160,7 @@ class ColumnarBatchSuite extends SparkFunSuite { * This test generates a random schema data, serializes it to column batches and verifies the * results. */ - def testRandomRows(flatSchema: Boolean, numFields: Int) { + def testRandomRows(flatSchema: Boolean, numFields: Int): Unit = { // TODO: Figure out why StringType doesn't work on jenkins. val types = Array( BooleanType, ByteType, FloatType, DoubleType, IntegerType, LongType, ShortType, 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 7fe00aef56e16..715534b0458dc 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 @@ -450,15 +450,6 @@ class JDBCSuite extends QueryTest urlWithUserAndPass, "TEST.PEOPLE", new Properties()).collect().length === 3) } - test("Basic API with illegal fetchsize") { - val properties = new Properties() - properties.setProperty(JDBCOptions.JDBC_BATCH_FETCH_SIZE, "-1") - val e = intercept[IllegalArgumentException] { - spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", properties).collect() - }.getMessage - assert(e.contains("Invalid value `-1` for parameter `fetchsize`")) - } - test("Missing partition columns") { withView("tempPeople") { val e = intercept[IllegalArgumentException] { @@ -743,7 +734,7 @@ class JDBCSuite extends QueryTest } test("compile filters") { - val compileFilter = PrivateMethod[Option[String]]('compileFilter) + val compileFilter = PrivateMethod[Option[String]](Symbol("compileFilter")) def doCompileFilter(f: Filter): String = JDBCRDD invokePrivate compileFilter(f, JdbcDialects.get("jdbc:")) getOrElse("") assert(doCompileFilter(EqualTo("col0", 3)) === """"col0" = 3""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 1ece98aa7eb3a..7c10f9950f8eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -26,7 +26,8 @@ import org.apache.spark.unsafe.types.UTF8String private[sql] abstract class DataSourceTest extends QueryTest { - protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: Boolean = false) { + protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], + enableRegex: Boolean = false): Unit = { test(sqlString) { withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> enableRegex.toString) { checkAnswer(spark.sql(sqlString), expectedAnswer) 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 0d236a43ece6b..9e33b8aaec5d9 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 @@ -634,6 +634,60 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } + test("Throw exceptions on inserting out-of-range int value with ANSI casting policy") { + withSQLConf( + SQLConf.STORE_ASSIGNMENT_POLICY.key -> SQLConf.StoreAssignmentPolicy.ANSI.toString) { + withTable("t") { + sql("create table t(b int) using parquet") + val outOfRangeValue1 = (Int.MaxValue + 1L).toString + var msg = intercept[SparkException] { + sql(s"insert into t values($outOfRangeValue1)") + }.getCause.getMessage + assert(msg.contains(s"Casting $outOfRangeValue1 to int causes overflow")) + + val outOfRangeValue2 = (Int.MinValue - 1L).toString + msg = intercept[SparkException] { + sql(s"insert into t values($outOfRangeValue2)") + }.getCause.getMessage + assert(msg.contains(s"Casting $outOfRangeValue2 to int causes overflow")) + } + } + } + + test("Throw exceptions on inserting out-of-range long value with ANSI casting policy") { + withSQLConf( + SQLConf.STORE_ASSIGNMENT_POLICY.key -> SQLConf.StoreAssignmentPolicy.ANSI.toString) { + withTable("t") { + sql("create table t(b long) using parquet") + val outOfRangeValue1 = Math.nextUp(Long.MaxValue) + var msg = intercept[SparkException] { + sql(s"insert into t values(${outOfRangeValue1}D)") + }.getCause.getMessage + assert(msg.contains(s"Casting $outOfRangeValue1 to long causes overflow")) + + val outOfRangeValue2 = Math.nextDown(Long.MinValue) + msg = intercept[SparkException] { + sql(s"insert into t values(${outOfRangeValue2}D)") + }.getCause.getMessage + assert(msg.contains(s"Casting $outOfRangeValue2 to long causes overflow")) + } + } + } + + test("Throw exceptions on inserting out-of-range decimal value with ANSI casting policy") { + withSQLConf( + SQLConf.STORE_ASSIGNMENT_POLICY.key -> SQLConf.StoreAssignmentPolicy.ANSI.toString) { + withTable("t") { + sql("create table t(b decimal(3,2)) using parquet") + val outOfRangeValue = "123.45" + val msg = intercept[SparkException] { + sql(s"insert into t values(${outOfRangeValue})") + }.getCause.getMessage + assert(msg.contains("cannot be represented as Decimal(3, 2)")) + } + } + } + test("SPARK-24860: dynamic partition overwrite specified per source without catalog table") { withTempPath { path => Seq((1, 1), (2, 2)).toDF("i", "part") @@ -675,7 +729,10 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { spark.sessionState.catalog.createTable(newTable, false) sql("INSERT INTO TABLE test_table SELECT 1, 'a'") - sql("INSERT INTO TABLE test_table SELECT 2, null") + val msg = intercept[AnalysisException] { + sql("INSERT INTO TABLE test_table SELECT 2, null") + }.getMessage + assert(msg.contains("Cannot write nullable values to non-null column 's'")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala index 87dce376a09dd..9b26a5659df49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala @@ -123,7 +123,8 @@ class PathOptionSuite extends DataSourceTest with SharedSparkSession { |USING ${classOf[TestOptionsSource].getCanonicalName} |OPTIONS (PATH '/tmp/path')""".stripMargin) sql("ALTER TABLE src SET LOCATION '/tmp/path2'") - assert(getPathOption("src").map(makeQualifiedPath) == Some(makeQualifiedPath("/tmp/path2"))) + assert(getPathOption("src") == + Some(CatalogUtils.URIToString(makeQualifiedPath("/tmp/path2")))) } withTable("src", "src2") { 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 7d343bb58ea3f..0b885c8429c65 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 @@ -22,10 +22,13 @@ import java.nio.file.Files import java.util.Locale import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.JobContext import org.apache.spark.SparkConf +import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.execution.DataSourceScanExec @@ -389,7 +392,7 @@ abstract class FileStreamSinkSuite extends StreamTest { var bytesWritten: Long = 0L try { spark.sparkContext.addSparkListener(new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { val outputMetrics = taskEnd.taskMetrics.outputMetrics recordsWritten += outputMetrics.recordsWritten bytesWritten += outputMetrics.bytesWritten @@ -473,6 +476,77 @@ abstract class FileStreamSinkSuite extends StreamTest { assert(outputFiles.toList.isEmpty, "Incomplete files should be cleaned up.") } } + + testQuietly("cleanup complete but invalid output for aborted job") { + withSQLConf(("spark.sql.streaming.commitProtocolClass", + classOf[PendingCommitFilesTrackingManifestFileCommitProtocol].getCanonicalName)) { + withTempDir { tempDir => + val checkpointDir = new File(tempDir, "chk") + val outputDir = new File(tempDir, "output @#output") + val inputData = MemoryStream[Int] + inputData.addData(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val q = inputData.toDS() + .repartition(10) + .map { value => + // we intend task failure after some tasks succeeds + if (value == 5) { + // put some delay to let other task commits before this task fails + Thread.sleep(100) + value / 0 + } else { + value + } + } + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("parquet") + .start(outputDir.getCanonicalPath) + + intercept[StreamingQueryException] { + try { + q.processAllAvailable() + } finally { + q.stop() + } + } + + import PendingCommitFilesTrackingManifestFileCommitProtocol._ + val outputFileNames = Files.walk(outputDir.toPath).iterator().asScala + .filter(_.toString.endsWith(".parquet")) + .map(_.getFileName.toString) + .toSet + val trackingFileNames = tracking.map(new Path(_).getName).toSet + + // there would be possible to have race condition: + // - some tasks complete while abortJob is being called + // we can't delete complete files for these tasks (it's OK since this is a best effort) + assert(outputFileNames.intersect(trackingFileNames).isEmpty, + "abortJob should clean up files reported as successful.") + } + } + } +} + +object PendingCommitFilesTrackingManifestFileCommitProtocol { + val tracking: ArrayBuffer[String] = new ArrayBuffer[String]() + + def cleanPendingCommitFiles(): Unit = tracking.clear() + def addPendingCommitFiles(paths: Seq[String]): Unit = tracking ++= paths +} + +class PendingCommitFilesTrackingManifestFileCommitProtocol(jobId: String, path: String) + extends ManifestFileCommitProtocol(jobId, path) { + import PendingCommitFilesTrackingManifestFileCommitProtocol._ + + override def setupJob(jobContext: JobContext): Unit = { + super.setupJob(jobContext) + cleanPendingCommitFiles() + } + + override def onTaskCommit(taskCommit: FileCommitProtocol.TaskCommitMessage): Unit = { + super.onTaskCommit(taskCommit) + addPendingCommitFiles(taskCommit.obj.asInstanceOf[Seq[SinkFileStatus]].map(_.path)) + } } class FileStreamSinkV1Suite extends FileStreamSinkSuite { 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 f59f819c9c108..01ec59f292309 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 @@ -49,7 +49,7 @@ abstract class FileStreamSourceTest * `FileStreamSource` actually being used in the execution. */ abstract class AddFileData extends AddData { - private val _qualifiedBasePath = PrivateMethod[Path]('qualifiedBasePath) + private val _qualifiedBasePath = PrivateMethod[Path](Symbol("qualifiedBasePath")) private def isSamePath(fileSource: FileStreamSource, srcPath: File): Boolean = { val path = (fileSource invokePrivate _qualifiedBasePath()).toString.stripPrefix("file:") @@ -178,7 +178,7 @@ abstract class FileStreamSourceTest } - protected def withTempDirs(body: (File, File) => Unit) { + protected def withTempDirs(body: (File, File) => Unit): Unit = { val src = Utils.createTempDir(namePrefix = "streaming.src") val tmp = Utils.createTempDir(namePrefix = "streaming.tmp") try { @@ -1218,8 +1218,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } test("compact interval metadata log") { - val _sources = PrivateMethod[Seq[Source]]('sources) - val _metadataLog = PrivateMethod[FileStreamSourceLog]('metadataLog) + val _sources = PrivateMethod[Seq[Source]](Symbol("sources")) + val _metadataLog = PrivateMethod[FileStreamSourceLog](Symbol("metadataLog")) def verify( execution: StreamExecution, @@ -1303,7 +1303,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { AddTextFileData("keep3", src, tmp), CheckAnswer("keep1", "keep2", "keep3"), AssertOnQuery("check getBatch") { execution: StreamExecution => - val _sources = PrivateMethod[Seq[Source]]('sources) + val _sources = PrivateMethod[Seq[Source]](Symbol("sources")) val fileSource = getSourcesFromStreamingQuery(execution).head def verify(startId: Option[Int], endId: Int, expected: String*): Unit = { 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 958d15ba1701d..75ad041ccb801 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 @@ -201,7 +201,7 @@ class StreamSuite extends StreamTest { } test("DataFrame reuse") { - def assertDF(df: DataFrame) { + def assertDF(df: DataFrame): Unit = { withTempDir { outputDir => withTempDir { checkpointDir => val query = df.writeStream.format("parquet") @@ -1175,7 +1175,7 @@ class FakeDefaultSource extends FakeSource { ds.toDF("a") } - override def stop() {} + override def stop(): Unit = {} } } } 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 b26d2556b2e36..09580b94056b4 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.streaming +import java.io.File import java.util.concurrent.CountDownLatch import scala.concurrent.Future @@ -28,7 +29,7 @@ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.Dataset +import org.apache.spark.sql.{Dataset, Encoders} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.streaming.util.BlockingSource @@ -242,6 +243,83 @@ class StreamingQueryManagerSuite extends StreamTest { } } + testQuietly("can't start a streaming query with the same name in the same session") { + val ds1 = makeDataset._2 + val ds2 = makeDataset._2 + val queryName = "abc" + + val query1 = ds1.writeStream.format("noop").queryName(queryName).start() + try { + val e = intercept[IllegalArgumentException] { + ds2.writeStream.format("noop").queryName(queryName).start() + } + assert(e.getMessage.contains("query with that name is already active")) + } finally { + query1.stop() + } + } + + testQuietly("can start a streaming query with the same name in a different session") { + val session2 = spark.cloneSession() + + val ds1 = MemoryStream(Encoders.INT, spark.sqlContext).toDS() + val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() + val queryName = "abc" + + val query1 = ds1.writeStream.format("noop").queryName(queryName).start() + val query2 = ds2.writeStream.format("noop").queryName(queryName).start() + + query1.stop() + query2.stop() + } + + testQuietly("can't start multiple instances of the same streaming query in the same session") { + withTempDir { dir => + val (ms1, ds1) = makeDataset + val (ms2, ds2) = makeDataset + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ds1.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + try { + val e = intercept[IllegalStateException] { + ds2.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + } + assert(e.getMessage.contains("same id")) + } finally { + query1.stop() + } + } + } + + testQuietly( + "can't start multiple instances of the same streaming query in the different sessions") { + withTempDir { dir => + val session2 = spark.cloneSession() + + val ms1 = MemoryStream(Encoders.INT, spark.sqlContext) + val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ms1.toDS().writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + try { + val e = intercept[IllegalStateException] { + ds2.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + } + assert(e.getMessage.contains("same id")) + } finally { + query1.stop() + } + } + } + /** Run a body of code by defining a query on each dataset */ private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = { failAfter(streamingTimeout) { 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 4a674b647917a..760731d26f051 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 @@ -29,7 +29,7 @@ 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.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.internal.Logging 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 5d6050dbda0fd..55b884573f647 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming.continuous import java.util.concurrent.{ArrayBlockingQueue, BlockingQueue} import org.mockito.Mockito._ -import org.scalatest.mockito.MockitoSugar +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rpc.RpcEndpointRef 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 48998f5d6c5e8..0e1c9b9c4ba46 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 @@ -21,7 +21,7 @@ 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 +import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.rpc.RpcEndpointRef 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 c630f1497a17e..f9fc540c2ab80 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 @@ -92,7 +92,7 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { spark.internalCreateDataFrame(spark.sparkContext.emptyRDD, schema, isStreaming = true) } - override def stop() {} + override def stop(): Unit = {} } } 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 67158fb99d13d..c1b29b5130e86 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 @@ -52,7 +52,7 @@ class BlockingSource extends StreamSourceProvider with StreamSinkProvider { import spark.implicits._ Seq[Int]().toDS().toDF() } - override def stop() {} + override def stop(): Unit = {} } } 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 01a03e484c90d..cef0e5ab47568 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 @@ -289,18 +289,20 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with 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() - assert(plan.isInstanceOf[AppendData]) + val e = intercept[AnalysisException] { + spark.range(10).write + .format(classOf[NoopDataSource].getName) + .save() + } + assert(e.getMessage.contains("ErrorIfExists")) - spark.range(10).write - .format(classOf[NoopDataSource].getName) - .mode("default") - .save() - sparkContext.listenerBus.waitUntilEmpty() - assert(plan.isInstanceOf[AppendData]) + val e2 = intercept[AnalysisException] { + spark.range(10).write + .format(classOf[NoopDataSource].getName) + .mode("default") + .save() + } + assert(e2.getMessage.contains("ErrorIfExists")) } finally { spark.listenerManager.unregister(listener) } 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 e96f05384c879..083b40d0680aa 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 @@ -22,8 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.sql.{functions, AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5b1352adddd89..9dd927084298a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -77,15 +77,6 @@ ${hive.group} hive-beeline
- - - ${hive.group} - hive-contrib - - - ${hive.group}.hcatalog - hive-hcatalog-core - org.eclipse.jetty jetty-server 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 36d4ac095e10c..9517a599be633 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 @@ -72,7 +72,7 @@ object HiveThriftServer2 extends Logging { server } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { // If the arguments contains "-h" or "--help", print out the usage and exit. if (args.contains("-h") || args.contains("--help")) { HiveServer2.main(args) @@ -303,7 +303,7 @@ private[hive] class HiveThriftServer2(sqlContext: SQLContext) // started, and then once only. private val started = new AtomicBoolean(false) - override def init(hiveConf: HiveConf) { + override def init(hiveConf: HiveConf): Unit = { val sparkSqlCliService = new SparkSQLCLIService(this, sqlContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala index 599294dfbb7d7..a4024be67ac9c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala @@ -18,11 +18,11 @@ package org.apache.spark.sql.hive.thriftserver private[hive] object ReflectionUtils { - def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { + def setSuperField(obj : Object, fieldName: String, fieldValue: Object): Unit = { setAncestorField(obj, 1, fieldName, fieldValue) } - def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { + def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef): Unit = { val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() val field = ancestor.getDeclaredField(fieldName) field.setAccessible(true) 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 f246f43435c75..68197a9de8566 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 @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.Utils import org.apache.hive.service.cli._ @@ -57,7 +58,8 @@ private[hive] class SparkExecuteStatementOperation( // This is only used when `spark.sql.thriftServer.incrementalCollect` is set to `false`. // In case of `true`, this will be `None` and FETCH_FIRST will trigger re-execution. private var resultList: Option[Array[SparkRow]] = _ - + private var previousFetchEndOffset: Long = 0 + private var previousFetchStartOffset: Long = 0 private var iter: Iterator[SparkRow] = _ private var dataTypes: Array[DataType] = _ private var statementId: String = _ @@ -78,7 +80,7 @@ private[hive] class SparkExecuteStatementOperation( HiveThriftServer2.listener.onOperationClosed(statementId) } - def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { + def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { dataTypes(ordinal) match { case StringType => to += from.getString(ordinal) @@ -113,14 +115,18 @@ private[hive] class SparkExecuteStatementOperation( } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = withSchedulerPool { + log.info(s"Received getNextRowSet request order=${order} and maxRowsL=${maxRowsL} " + + s"with ${statementId}") validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) val resultRowSet: RowSet = ThriftserverShimUtils.resultRowSet(getResultSetSchema, getProtocolVersion) - // Reset iter to header when fetching start from first row - if (order.equals(FetchOrientation.FETCH_FIRST)) { + // Reset iter when FETCH_FIRST or FETCH_PRIOR + if ((order.equals(FetchOrientation.FETCH_FIRST) || + order.equals(FetchOrientation.FETCH_PRIOR)) && previousFetchEndOffset != 0) { + // Reset the iterator to the beginning of the query. iter = if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None result.toLocalIterator.asScala @@ -132,6 +138,28 @@ private[hive] class SparkExecuteStatementOperation( } } + var resultOffset = { + if (order.equals(FetchOrientation.FETCH_FIRST)) { + logInfo(s"FETCH_FIRST request with $statementId. Resetting to resultOffset=0") + 0 + } else if (order.equals(FetchOrientation.FETCH_PRIOR)) { + // TODO: FETCH_PRIOR should be handled more efficiently than rewinding to beginning and + // reiterating. + val targetOffset = math.max(previousFetchStartOffset - maxRowsL, 0) + logInfo(s"FETCH_PRIOR request with $statementId. Resetting to resultOffset=$targetOffset") + var off = 0 + while (off < targetOffset && iter.hasNext) { + iter.next() + off += 1 + } + off + } else { // FETCH_NEXT + previousFetchEndOffset + } + } + + resultRowSet.setStartOffset(resultOffset) + previousFetchStartOffset = resultOffset if (!iter.hasNext) { resultRowSet } else { @@ -152,7 +180,11 @@ private[hive] class SparkExecuteStatementOperation( } resultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) curRow += 1 + resultOffset += 1 } + previousFetchEndOffset = resultOffset + log.info(s"Returning result set with ${curRow} rows from offsets " + + s"[$previousFetchStartOffset, $previousFetchEndOffset) with $statementId") resultRowSet } } @@ -267,6 +299,13 @@ private[hive] class SparkExecuteStatementOperation( // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => + // When cancel() or close() is called very quickly after the query is started, + // then they may both call cleanup() before Spark Jobs are started. But before background + // task interrupted, it may have start some spark job, so we need to cancel again to + // make sure job was cancelled when background thread was interrupted + if (statementId != null) { + sqlContext.sparkContext.cancelJobGroup(statementId) + } val currentState = getStatus().getState() if (currentState.isTerminal) { // This may happen if the execution was cancelled, and then closed from another thread. @@ -274,12 +313,16 @@ private[hive] class SparkExecuteStatementOperation( } else { logError(s"Error executing query with $statementId, currentState $currentState, ", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - if (e.isInstanceOf[HiveSQLException]) { - throw e.asInstanceOf[HiveSQLException] - } else { - throw new HiveSQLException("Error running query: " + e.toString, e) + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error running query: " + root.toString, root) } } } finally { @@ -303,7 +346,7 @@ private[hive] class SparkExecuteStatementOperation( } } - private def cleanup(state: OperationState) { + private def cleanup(state: OperationState): Unit = { setState(state) if (runInBackground) { val backgroundHandle = getBackgroundHandle() diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index cde99fd35bd59..6c8a5b00992da 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.GetCatalogsOperation @@ -68,11 +69,20 @@ private[hive] class SparkGetCatalogsOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get catalogs operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting catalogs: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } 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 index 89faff2f6f913..f845a2285b9a3 100644 --- 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 @@ -22,6 +22,7 @@ import java.util.regex.Pattern import scala.collection.JavaConverters.seqAsJavaListConverter +import org.apache.commons.lang3.exception.ExceptionUtils 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._ @@ -129,11 +130,20 @@ private[hive] class SparkGetColumnsOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get columns operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting columns: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 462e57300e82b..1cdd8918421bb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -22,6 +22,7 @@ import java.util.UUID import scala.collection.JavaConverters.seqAsJavaListConverter +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetFunctionsOperation @@ -104,11 +105,20 @@ private[hive] class SparkGetFunctionsOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get functions operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting functions: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } 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 index 87ef154bcc8ab..928610a6bcff9 100644 --- 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID import java.util.regex.Pattern +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetSchemasOperation @@ -87,11 +88,20 @@ private[hive] class SparkGetSchemasOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get schemas operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting schemas: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index 8f2257f77d2a0..ec03f1e148e69 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTableTypesOperation @@ -74,11 +75,20 @@ private[hive] class SparkGetTableTypesOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get table types operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting table types: " + root.toString, root) + } } 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 index 6441dc50f49fe..bf9cf7ad46d95 100644 --- 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 @@ -22,6 +22,7 @@ import java.util.regex.Pattern import scala.collection.JavaConverters._ +import org.apache.commons.lang3.exception.ExceptionUtils 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._ @@ -30,7 +31,6 @@ 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} @@ -119,11 +119,20 @@ private[hive] class SparkGetTablesOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get tables operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting tables: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 7a6a8c59b7216..0d263b09d57d3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.GetTypeInfoOperation @@ -92,11 +93,20 @@ private[hive] class SparkGetTypeInfoOperation( }) setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get type info with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting type info: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } 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 9f554b200f775..b665d4a31b9b1 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,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ import java.nio.charset.StandardCharsets.UTF_8 -import java.util.{ArrayList => JArrayList, Locale} +import java.util.{ArrayList => JArrayList, List => JList, Locale} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ @@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.log4j.Level import org.apache.thrift.transport.TSocket +import sun.misc.{Signal, SignalHandler} import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil @@ -63,7 +64,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while * a command is being processed by the current thread. */ - def installSignalHandler() { + def installSignalHandler(): Unit = { HiveInterruptUtils.add(() => { // Handle remote execution mode if (SparkSQLEnv.sparkContext != null) { @@ -77,7 +78,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { }) } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val oproc = new OptionsProcessor() if (!oproc.process_stage1(args)) { System.exit(1) @@ -165,6 +166,13 @@ private[hive] object SparkSQLCLIDriver extends Logging { StringUtils.split(auxJars, ",").foreach(resourceLoader.addJar(_)) } + // The class loader of CliSessionState's conf is current main thread's class loader + // used to load jars passed by --jars. One class loader used by AddJarCommand is + // sharedState.jarClassLoader which contain jar path passed by --jars in main thread. + // We set CliSessionState's conf class loader to sharedState.jarClassLoader. + // Thus we can load all jars passed by --jars and AddJarCommand. + sessionState.getConf.setClassLoader(SparkSQLEnv.sqlContext.sharedState.jarClassLoader) + // TODO work around for set the log output to console, because the HiveContext // will set the output into an invalid buffer. sessionState.in = System.in @@ -431,5 +439,112 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { ret } } + + // Adapted processLine from Hive 2.3's CliDriver.processLine. + override def processLine(line: String, allowInterrupting: Boolean): Int = { + var oldSignal: SignalHandler = null + var interruptSignal: Signal = null + + if (allowInterrupting) { + // Remember all threads that were running at the time we started line processing. + // Hook up the custom Ctrl+C handler while processing this line + interruptSignal = new Signal("INT") + oldSignal = Signal.handle(interruptSignal, new SignalHandler() { + private var interruptRequested: Boolean = false + + override def handle(signal: Signal) { + val initialRequest = !interruptRequested + interruptRequested = true + + // Kill the VM on second ctrl+c + if (!initialRequest) { + console.printInfo("Exiting the JVM") + System.exit(127) + } + + // Interrupt the CLI thread to stop the current statement and return + // to prompt + console.printInfo("Interrupting... Be patient, this might take some time.") + console.printInfo("Press Ctrl+C again to kill JVM") + + HiveInterruptUtils.interrupt() + } + }) + } + + try { + var lastRet: Int = 0 + + // we can not use "split" function directly as ";" may be quoted + val commands = splitSemiColon(line).asScala + var command: String = "" + for (oneCmd <- commands) { + if (StringUtils.endsWith(oneCmd, "\\")) { + command += StringUtils.chop(oneCmd) + ";" + } else { + command += oneCmd + if (!StringUtils.isBlank(command)) { + val ret = processCmd(command) + command = "" + lastRet = ret + val ignoreErrors = HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS) + if (ret != 0 && !ignoreErrors) { + CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf]) + ret + } + } + } + } + CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf]) + lastRet + } finally { + // Once we are done processing the line, restore the old handler + if (oldSignal != null && interruptSignal != null) { + Signal.handle(interruptSignal, oldSignal) + } + } + } + + // Adapted splitSemiColon from Hive 2.3's CliDriver.splitSemiColon. + private def splitSemiColon(line: String): JList[String] = { + var insideSingleQuote = false + var insideDoubleQuote = false + var escape = false + var beginIndex = 0 + val ret = new JArrayList[String] + for (index <- 0 until line.length) { + if (line.charAt(index) == '\'') { + // take a look to see if it is escaped + if (!escape) { + // flip the boolean variable + insideSingleQuote = !insideSingleQuote + } + } else if (line.charAt(index) == '\"') { + // take a look to see if it is escaped + if (!escape) { + // flip the boolean variable + insideDoubleQuote = !insideDoubleQuote + } + } else if (line.charAt(index) == ';') { + if (insideSingleQuote || insideDoubleQuote) { + // do not split + } else { + // split, do not include ; itself + ret.add(line.substring(beginIndex, index)) + beginIndex = index + 1 + } + } else { + // nothing to do + } + // set the escape + if (escape) { + escape = false + } else if (line.charAt(index) == '\\') { + escape = true + } + } + ret.add(line.substring(beginIndex)) + ret + } } 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 c32d908ad1bba..1644ecb2453be 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 @@ -43,7 +43,7 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC extends CLIService(hiveServer) with ReflectedCompositeService { - override def init(hiveConf: HiveConf) { + override def init(hiveConf: HiveConf): Unit = { setSuperField(this, "hiveConf", hiveConf) val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, sqlContext) @@ -105,7 +105,7 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC } private[thriftserver] trait ReflectedCompositeService { this: AbstractService => - def initCompositeService(hiveConf: HiveConf) { + def initCompositeService(hiveConf: HiveConf): Unit = { // Emulating `CompositeService.init(hiveConf)` val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") serviceList.asScala.foreach(_.init(hiveConf)) 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 960fdd11db15d..362ac362e9718 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 @@ -94,7 +94,7 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont override def getSchema: Schema = tableSchema - override def destroy() { + override def destroy(): Unit = { super.destroy() hiveResponse = null tableSchema = null diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 674da18ca1803..2fda9d0a4f60f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -33,7 +33,7 @@ private[hive] object SparkSQLEnv extends Logging { var sqlContext: SQLContext = _ var sparkContext: SparkContext = _ - def init() { + def init(): Unit = { if (sqlContext == null) { val sparkConf = new SparkConf(loadDefaults = true) // If user doesn't specify the appName, we want to get [SparkSQL::localHostName] instead of @@ -60,7 +60,7 @@ private[hive] object SparkSQLEnv extends Logging { } /** Cleans up and shuts down the Spark SQL environments. */ - def stop() { + def stop(): Unit = { logDebug("Shutting down Spark SQL Environment") // Stop the SparkContext if (SparkSQLEnv.sparkContext != null) { 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 8369c09b09e74..c4248bfde38cc 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 @@ -38,7 +38,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: private lazy val sparkSqlOperationManager = new SparkSQLOperationManager() - override def init(hiveConf: HiveConf) { + override def init(hiveConf: HiveConf): Unit = { setSuperField(this, "operationManager", sparkSqlOperationManager) super.init(hiveConf) } @@ -73,7 +73,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: sessionHandle } - override def closeSession(sessionHandle: SessionHandle) { + override def closeSession(sessionHandle: SessionHandle): Unit = { HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) 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 1a7db1b58e5bd..e472aaad5bdc6 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 @@ -17,18 +17,22 @@ package org.apache.spark.sql.hive.thriftserver.ui +import java.net.URLEncoder +import java.nio.charset.StandardCharsets.UTF_8 import java.util.Calendar import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.collection.JavaConverters._ +import scala.xml.{Node, Unparsed} import org.apache.commons.text.StringEscapeUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState, SessionInfo} +import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ - +import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of the thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { @@ -68,41 +72,56 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch statements of the thrift server program */ 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", "Close Time", - "Execution Time", "Duration", "Statement", "State", "Detail") - val dataRows = listener.getExecutionList.sortBy(_.startTimestamp).reverse - - def generateDataRow(info: ExecutionInfo): Seq[Node] = { - val jobLink = info.jobId.map { id: String => - - [{id}] - + + val sqlTableTag = "sqlstat" + + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(sqlTableTag)) + .map { case (name, vals) => + name + "=" + vals(0) } - val detail = Option(info.detail).filter(!_.isEmpty).getOrElse(info.executePlan) -
{info.userName} - {jobLink} - {info.groupId}{formatDate(info.startTimestamp)}{if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)}{if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} - {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} - {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement}{info.state}
{errorSummary}{details} + + + {header} {Unparsed(arrow)} + + + + + {header} {Unparsed(arrow)} + + + + + {header} + + + + + {header} + +
+ {info.userName} + + {jobLinks(sqlStatsTableRow.jobId)} + + {info.groupId} + + {UIUtils.formatDate(startTime)} + + {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} + + {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} + + {UIUtils.formatDuration(executionTime)} + + {UIUtils.formatDuration(duration)} + + {info.statement} + + {info.state} +
+ {errorSummary}{details} +