From 729b3180bcbaa5289cb9a5848a3cce9010e75515 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 15 Sep 2019 10:15:49 -0700 Subject: [PATCH 01/87] [SPARK-29087][CORE][STREAMING] Use DelegatingServletContextHandler to avoid CCE ### What changes were proposed in this pull request? [SPARK-27122](https://github.com/apache/spark/pull/24088) fixes `ClassCastException` at `yarn` module by introducing `DelegatingServletContextHandler`. Initially, this was discovered with JDK9+, but the class path issues affected JDK8 environment, too. After [SPARK-28709](https://github.com/apache/spark/pull/25439), I also hit the similar issue at `streaming` module. This PR aims to fix `streaming` module by adding `getContextPath` to `DelegatingServletContextHandler` and using it. ### Why are the changes needed? Currently, when we test `streaming` module independently, it fails like the following. ``` $ build/mvn test -pl streaming ... UISeleniumSuite: - attaching and detaching a Streaming tab *** FAILED *** java.lang.ClassCastException: org.sparkproject.jetty.servlet.ServletContextHandler cannot be cast to org.eclipse.jetty.servlet.ServletContextHandler ... Tests: succeeded 337, failed 1, canceled 0, ignored 1, pending 0 *** 1 TEST FAILED *** [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the modified tests. And do the following manually. Since you can observe this when you run `streaming` module test only (instead of running all), you need to install the changed `core` module and use it. ``` $ java -version openjdk version "1.8.0_222" OpenJDK Runtime Environment (AdoptOpenJDK)(build 1.8.0_222-b10) OpenJDK 64-Bit Server VM (AdoptOpenJDK)(build 25.222-b10, mixed mode) $ build/mvn install -DskipTests $ build/mvn test -pl streaming ``` Closes #25791 from dongjoon-hyun/SPARK-29087. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ui/WebUI.scala | 4 ++++ .../scala/org/apache/spark/streaming/UISeleniumSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 1fe822a0e3b57..39d64056ad430 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -236,4 +236,8 @@ private[spark] class DelegatingServletContextHandler(handler: ServletContextHand def filterCount(): Int = { handler.getServletHandler.getFilters.length } + + def getContextPath(): String = { + handler.getContextPath + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 1d34221fde4f4..3d477ca3d0a9d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -97,7 +97,7 @@ class UISeleniumSuite val sparkUI = ssc.sparkContext.ui.get - sparkUI.getHandlers.count(_.getContextPath.contains("/streaming")) should be (5) + sparkUI.getDelegatingHandlers.count(_.getContextPath.contains("/streaming")) should be (5) eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/")) @@ -198,7 +198,7 @@ class UISeleniumSuite ssc.stop(false) - sparkUI.getHandlers.count(_.getContextPath.contains("/streaming")) should be (0) + sparkUI.getDelegatingHandlers.count(_.getContextPath.contains("/streaming")) should be (0) eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/")) From 1b7afc0c986ed8e5431df351f51434424460f4b3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Sep 2019 11:02:15 -0700 Subject: [PATCH 02/87] [SPARK-28471][SQL][DOC][FOLLOWUP] Fix year patterns in the comments of date-time expressions ### What changes were proposed in this pull request? In the PR, I propose to fix comments of date-time expressions, and replace the `yyyy` pattern by `uuuu` when the implementation supposes the former one. ### Why are the changes needed? To make comments consistent to implementations. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running Scala Style checker. Closes #25796 from MaxGekk/year-pattern-uuuu-followup. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 2 +- .../apache/spark/sql/catalyst/util/TimestampFormatter.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) 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 9d43701f03056..eafdca2bfc53e 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 @@ -647,7 +647,7 @@ case class ToUnixTimestamp( * Converts time string with given pattern to Unix time stamp (in seconds), returns null if fail. * See [https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html]. * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. - * If the second parameter is missing, use "yyyy-MM-dd HH:mm:ss". + * If the second parameter is missing, use "uuuu-MM-dd HH:mm:ss". * If no parameters provided, the first parameter will be current_timestamp. * If the first parameter is a Date or Timestamp instead of String, we will ignore the * second parameter. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index b23cec64568df..3dfcf1a33eb56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -67,7 +67,7 @@ class Iso8601TimestampFormatter( } /** - * The formatter parses/formats timestamps according to the pattern `yyyy-MM-dd HH:mm:ss.[..fff..]` + * The formatter parses/formats timestamps according to the pattern `uuuu-MM-dd HH:mm:ss.[..fff..]` * where `[..fff..]` is a fraction of second up to microsecond resolution. The formatter does not * output trailing zeros in the fraction. For example, the timestamp `2019-03-05 15:00:01.123400` is * formatted as the string `2019-03-05 15:00:01.1234`. 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 395f1b4667b1c..84e0eaff2d428 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 @@ -2634,8 +2634,8 @@ object functions { * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param dateExpr A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param format A pattern `dd.MM.yyyy` would return a string like `18.03.1993` + * can be cast to a timestamp, such as `uuuu-MM-dd` or `uuuu-MM-dd HH:mm:ss.SSSS` + * @param format A pattern `dd.MM.uuuu` would return a string like `18.03.1993` * @return A string, or null if `dateExpr` was a string that could not be cast to a timestamp * @note Use specialized functions like [[year]] whenever possible as they benefit from a * specialized implementation. From 61e5aebce3925e7c512899939688e0eee4ac8a06 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 15 Sep 2019 11:04:56 -0700 Subject: [PATCH 03/87] [SPARK-29046][SQL] Fix NPE in SQLConf.get when active SparkContext is stopping ### What changes were proposed in this pull request? This patch fixes the bug regarding NPE in SQLConf.get, which is only possible when SparkContext._dagScheduler is null due to stopping SparkContext. The logic doesn't seem to consider active SparkContext could be in progress of stopping. Note that it can't be encountered easily as SparkContext.stop() blocks the main thread, but there're many cases which SQLConf.get is accessed concurrently while SparkContext.stop() is executing - users run another threads, or listener is accessing SQLConf.get after dagScheduler is set to null (this is the case what I encountered.) ### Why are the changes needed? The bug brings NPE. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Previous patch #25753 was tested with new UT, and due to disruption with other tests in concurrent test run, the test is excluded in this patch. Closes #25790 from HeartSaVioR/SPARK-29046-v2. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 d9b0a72618c7e..4f3e39ad49afe 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 @@ -139,7 +139,8 @@ object SQLConf { } } else { val isSchedulerEventLoopThread = SparkContext.getActive - .map(_.dagScheduler.eventProcessLoop.eventThread) + .flatMap { sc => Option(sc.dagScheduler) } + .map(_.eventProcessLoop.eventThread) .exists(_.getId == Thread.currentThread().getId) if (isSchedulerEventLoopThread) { // DAGScheduler event loop thread does not have an active SparkSession, the `confGetter` From b91648cfd0d7ab7014a137cdb61d8dbb3611438c Mon Sep 17 00:00:00 2001 From: "changchun.wang" <251922566@qq.com> Date: Sun, 15 Sep 2019 11:11:38 -0700 Subject: [PATCH 04/87] [SPARK-28856][FOLLOW-UP][SQL][TEST] Add the `namespaces` keyword to TableIdentifierParserSuite ### What changes were proposed in this pull request? This PR add the `namespaces` keyword to `TableIdentifierParserSuite`. ### Why are the changes needed? Improve the test. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25758 from highmoutain/3.0bugfix. Authored-by: changchun.wang <251922566@qq.com> Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/TableIdentifierParserSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 246d0cf01e9ba..81f0bd024a80e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -443,6 +443,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "month", "months", "msck", + "namespaces", "natural", "no", "not", From 7d4eb38bbcc887fb61ba7344df3f77a046ad77f8 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sun, 15 Sep 2019 11:17:30 -0700 Subject: [PATCH 05/87] [SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Currently, there is no migration section for PySpark, SparkCore and Structured Streaming. It is difficult for users to know what to do when they upgrade. This PR proposes to create create a "Migration Guide" tap at Spark documentation. ![Screen Shot 2019-09-11 at 7 02 05 PM](https://user-images.githubusercontent.com/6477701/64688126-ad712f80-d4c6-11e9-8672-9a2c56c05bf8.png) ![Screen Shot 2019-09-11 at 7 27 15 PM](https://user-images.githubusercontent.com/6477701/64689915-389ff480-d4ca-11e9-8c54-7f46095d0d23.png) This page will contain migration guides for Spark SQL, PySpark, SparkR, MLlib, Structured Streaming and Core. Basically it is a refactoring. There are some new information added, which I will leave a comment inlined for easier review. 1. **MLlib** Merge [ml-guide.html#migration-guide](https://spark.apache.org/docs/latest/ml-guide.html#migration-guide) and [ml-migration-guides.html](https://spark.apache.org/docs/latest/ml-migration-guides.html) ``` 'docs/ml-guide.md' ↓ Merge new/old migration guides 'docs/ml-migration-guide.md' ``` 2. **PySpark** Extract PySpark specific items from https://spark.apache.org/docs/latest/sql-migration-guide-upgrade.html ``` 'docs/sql-migration-guide-upgrade.md' ↓ Extract PySpark specific items 'docs/pyspark-migration-guide.md' ``` 3. **SparkR** Move [sparkr.html#migration-guide](https://spark.apache.org/docs/latest/sparkr.html#migration-guide) into a separate file, and extract from [sql-migration-guide-upgrade.html](https://spark.apache.org/docs/latest/sql-migration-guide-upgrade.html) ``` 'docs/sparkr.md' 'docs/sql-migration-guide-upgrade.md' Move migration guide section ↘ ↙ Extract SparkR specific items docs/sparkr-migration-guide.md ``` 4. **Core** Newly created at `'docs/core-migration-guide.md'`. I skimmed resolved JIRAs at 3.0.0 and found some items to note. 5. **Structured Streaming** Newly created at `'docs/ss-migration-guide.md'`. I skimmed resolved JIRAs at 3.0.0 and found some items to note. 6. **SQL** Merged [sql-migration-guide-upgrade.html](https://spark.apache.org/docs/latest/sql-migration-guide-upgrade.html) and [sql-migration-guide-hive-compatibility.html](https://spark.apache.org/docs/latest/sql-migration-guide-hive-compatibility.html) ``` 'docs/sql-migration-guide-hive-compatibility.md' 'docs/sql-migration-guide-upgrade.md' Move Hive compatibility section ↘ ↙ Left over after filtering PySpark and SparkR items 'docs/sql-migration-guide.md' ``` ### Why are the changes needed? In order for users in production to effectively migrate to higher versions, and detect behaviour or breaking changes before upgrading and/or migrating. ### Does this PR introduce any user-facing change? Yes, this changes Spark's documentation at https://spark.apache.org/docs/latest/index.html. ### How was this patch tested? Manually build the doc. This can be verified as below: ```bash cd docs SKIP_API=1 jekyll build open _site/index.html ``` Closes #25757 from HyukjinKwon/migration-doc. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- docs/_data/menu-migration.yaml | 12 + docs/_data/menu-sql.yaml | 10 +- .../_includes/nav-left-wrapper-migration.html | 6 + docs/_layouts/global.html | 7 +- docs/core-migration-guide.md | 32 + docs/index.md | 1 + docs/migration-guide.md | 30 + docs/ml-guide.md | 65 +- ...ration-guides.md => ml-migration-guide.md} | 96 +- docs/pyspark-migration-guide.md | 120 +++ docs/sparkr-migration-guide.md | 77 ++ docs/sparkr.md | 46 +- .../sql-migration-guide-hive-compatibility.md | 167 ---- docs/sql-migration-guide-upgrade.md | 834 ---------------- docs/sql-migration-guide.md | 910 +++++++++++++++++- ...gration-guides.md => sql-migration-old.md} | 7 +- docs/ss-migration-guide.md | 32 + 17 files changed, 1295 insertions(+), 1157 deletions(-) create mode 100644 docs/_data/menu-migration.yaml create mode 100644 docs/_includes/nav-left-wrapper-migration.html create mode 100644 docs/core-migration-guide.md create mode 100644 docs/migration-guide.md rename docs/{ml-migration-guides.md => ml-migration-guide.md} (85%) create mode 100644 docs/pyspark-migration-guide.md create mode 100644 docs/sparkr-migration-guide.md delete mode 100644 docs/sql-migration-guide-hive-compatibility.md delete mode 100644 docs/sql-migration-guide-upgrade.md rename docs/{mllib-migration-guides.md => sql-migration-old.md} (73%) create mode 100644 docs/ss-migration-guide.md diff --git a/docs/_data/menu-migration.yaml b/docs/_data/menu-migration.yaml new file mode 100644 index 0000000000000..1d8b311dd64fb --- /dev/null +++ b/docs/_data/menu-migration.yaml @@ -0,0 +1,12 @@ +- text: Spark Core + url: core-migration-guide.html +- text: SQL, Datasets and DataFrame + url: sql-migration-guide.html +- text: Structured Streaming + url: ss-migration-guide.html +- text: MLlib (Machine Learning) + url: ml-migration-guide.html +- text: PySpark (Python on Spark) + url: pyspark-migration-guide.html +- text: SparkR (R on Spark) + url: sparkr-migration-guide.html diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index d03616163c964..07ca150a1523f 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -64,15 +64,7 @@ - text: Usage Notes url: sql-pyspark-pandas-with-arrow.html#usage-notes - text: Migration Guide - url: sql-migration-guide.html - subitems: - - text: Spark SQL Upgrading Guide - url: sql-migration-guide-upgrade.html - - text: Compatibility with Apache Hive - url: sql-migration-guide-hive-compatibility.html - - text: SQL Reserved/Non-Reserved Keywords - url: sql-reserved-and-non-reserved-keywords.html - + url: sql-migration-old.html - text: SQL Reference url: sql-ref.html subitems: diff --git a/docs/_includes/nav-left-wrapper-migration.html b/docs/_includes/nav-left-wrapper-migration.html new file mode 100644 index 0000000000000..4318a324a9475 --- /dev/null +++ b/docs/_includes/nav-left-wrapper-migration.html @@ -0,0 +1,6 @@ +
+
+

Migration Guide

+ {% include nav-left.html nav=include.nav-migration %} +
+
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 8ea15dc71d541..d5fb18bfb06c0 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -112,6 +112,7 @@
  • Job Scheduling
  • Security
  • Hardware Provisioning
  • +
  • Migration Guide
  • Building Spark
  • Contributing to Spark
  • @@ -126,8 +127,10 @@
    - {% if page.url contains "/ml" or page.url contains "/sql" %} - {% if page.url contains "/ml" %} + {% if page.url contains "/ml" or page.url contains "/sql" or page.url contains "migration-guide.html" %} + {% if page.url contains "migration-guide.html" %} + {% include nav-left-wrapper-migration.html nav-migration=site.data.menu-migration %} + {% elsif page.url contains "/ml" %} {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %} {% else %} {% include nav-left-wrapper-sql.html nav-sql=site.data.menu-sql %} diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md new file mode 100644 index 0000000000000..1c47dfb27124c --- /dev/null +++ b/docs/core-migration-guide.md @@ -0,0 +1,32 @@ +--- +layout: global +title: "Migration Guide: Spark Core" +displayTitle: "Migration Guide: Spark Core" +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +* Table of contents +{:toc} + +## Upgrading from Core 2.4 to 3.0 + +- In Spark 3.0, deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. + +- In Spark 3.0, deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. + +- In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. + diff --git a/docs/index.md b/docs/index.md index 4217918a87462..6cfbbaed20028 100644 --- a/docs/index.md +++ b/docs/index.md @@ -146,6 +146,7 @@ options for deployment: * Integration with other storage systems: * [Cloud Infrastructures](cloud-integration.html) * [OpenStack Swift](storage-openstack-swift.html) +* [Migration Guide](migration-guide.html): Migration guides for Spark components * [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](https://spark.apache.org/contributing.html) * [Third Party Projects](https://spark.apache.org/third-party-projects.html): related third party Spark projects diff --git a/docs/migration-guide.md b/docs/migration-guide.md new file mode 100644 index 0000000000000..9ca0ada37a2fe --- /dev/null +++ b/docs/migration-guide.md @@ -0,0 +1,30 @@ +--- +layout: global +title: Migration Guide +displayTitle: Migration Guide +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +This page documents sections of the migration guide for each component in order +for users to migrate effectively. + +* [Spark Core](core-migration-guide.html) +* [SQL, Datasets, and DataFrame](sql-migration-guide.html) +* [Structured Streaming](ss-migration-guide.html) +* [MLlib (Machine Learning)](ml-migration-guide.html) +* [PySpark (Python on Spark)](pyspark-migration-guide.html) +* [SparkR (R on Spark)](sparkr-migration-guide.html) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 4661d6cd87c04..7b4fa4f651e64 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -113,68 +113,7 @@ transforming multiple columns. * Robust linear regression with Huber loss ([SPARK-3181](https://issues.apache.org/jira/browse/SPARK-3181)). -# Migration guide +# Migration Guide -MLlib is under active development. -The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and the migration guide below will explain all changes between releases. +The migration guide is now archived [on this page](ml-migration-guide.html). -## From 2.4 to 3.0 - -### Breaking changes - -* `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. - -### Changes of behavior - -* [SPARK-11215](https://issues.apache.org/jira/browse/SPARK-11215): - In Spark 2.4 and previous versions, when specifying `frequencyDesc` or `frequencyAsc` as - `stringOrderType` param in `StringIndexer`, in case of equal frequency, the order of - strings is undefined. Since Spark 3.0, the strings with equal frequency are further - sorted by alphabet. And since Spark 3.0, `StringIndexer` supports encoding multiple - columns. - -## From 2.2 to 2.3 - -### Breaking changes - -* The class and trait hierarchy for logistic regression model summaries was changed to be cleaner -and better accommodate the addition of the multi-class summary. This is a breaking change for user -code that casts a `LogisticRegressionTrainingSummary` to a -`BinaryLogisticRegressionTrainingSummary`. Users should instead use the `model.binarySummary` -method. See [SPARK-17139](https://issues.apache.org/jira/browse/SPARK-17139) for more detail -(_note_ this is an `Experimental` API). This _does not_ affect the Python `summary` method, which -will still work correctly for both multinomial and binary cases. - -### Deprecations and changes of behavior - -**Deprecations** - -* `OneHotEncoder` has been deprecated and will be removed in `3.0`. It has been replaced by the -new [`OneHotEncoderEstimator`](ml-features.html#onehotencoderestimator) -(see [SPARK-13030](https://issues.apache.org/jira/browse/SPARK-13030)). **Note** that -`OneHotEncoderEstimator` will be renamed to `OneHotEncoder` in `3.0` (but -`OneHotEncoderEstimator` will be kept as an alias). - -**Changes of behavior** - -* [SPARK-21027](https://issues.apache.org/jira/browse/SPARK-21027): - The default parallelism used in `OneVsRest` is now set to 1 (i.e. serial). In `2.2` and - earlier versions, the level of parallelism was set to the default threadpool size in Scala. -* [SPARK-22156](https://issues.apache.org/jira/browse/SPARK-22156): - The learning rate update for `Word2Vec` was incorrect when `numIterations` was set greater than - `1`. This will cause training results to be different between `2.3` and earlier versions. -* [SPARK-21681](https://issues.apache.org/jira/browse/SPARK-21681): - Fixed an edge case bug in multinomial logistic regression that resulted in incorrect coefficients - when some features had zero variance. -* [SPARK-16957](https://issues.apache.org/jira/browse/SPARK-16957): - Tree algorithms now use mid-points for split values. This may change results from model training. -* [SPARK-14657](https://issues.apache.org/jira/browse/SPARK-14657): - Fixed an issue where the features generated by `RFormula` without an intercept were inconsistent - with the output in R. This may change results from model training in this scenario. - -## Previous Spark versions - -Earlier migration guides are archived [on this page](ml-migration-guides.html). - ---- diff --git a/docs/ml-migration-guides.md b/docs/ml-migration-guide.md similarity index 85% rename from docs/ml-migration-guides.md rename to docs/ml-migration-guide.md index 99edd9bd69efa..9e8cd3e07b1ee 100644 --- a/docs/ml-migration-guides.md +++ b/docs/ml-migration-guide.md @@ -1,8 +1,7 @@ --- layout: global -title: Old Migration Guides - MLlib -displayTitle: Old Migration Guides - MLlib -description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +title: "Migration Guide: MLlib (Machine Learning)" +displayTitle: "Migration Guide: MLlib (Machine Learning)" license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -20,15 +19,80 @@ license: | limitations under the License. --- -The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). +* Table of contents +{:toc} -## From 2.1 to 2.2 +Note that this migration guide describes the items specific to MLlib. +Many items of SQL migration can be applied when migrating MLlib to higher versions for DataFrame-based APIs. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from MLlib 2.4 to 3.0 + +### Breaking changes +{:.no_toc} + +* `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. + +### Changes of behavior +{:.no_toc} + +* [SPARK-11215](https://issues.apache.org/jira/browse/SPARK-11215): + In Spark 2.4 and previous versions, when specifying `frequencyDesc` or `frequencyAsc` as + `stringOrderType` param in `StringIndexer`, in case of equal frequency, the order of + strings is undefined. Since Spark 3.0, the strings with equal frequency are further + sorted by alphabet. And since Spark 3.0, `StringIndexer` supports encoding multiple + columns. + +## Upgrading from MLlib 2.2 to 2.3 + +### Breaking changes +{:.no_toc} + +* The class and trait hierarchy for logistic regression model summaries was changed to be cleaner +and better accommodate the addition of the multi-class summary. This is a breaking change for user +code that casts a `LogisticRegressionTrainingSummary` to a +`BinaryLogisticRegressionTrainingSummary`. Users should instead use the `model.binarySummary` +method. See [SPARK-17139](https://issues.apache.org/jira/browse/SPARK-17139) for more detail +(_note_ this is an `Experimental` API). This _does not_ affect the Python `summary` method, which +will still work correctly for both multinomial and binary cases. + +### Deprecations and changes of behavior +{:.no_toc} + +**Deprecations** + +* `OneHotEncoder` has been deprecated and will be removed in `3.0`. It has been replaced by the +new [`OneHotEncoderEstimator`](ml-features.html#onehotencoderestimator) +(see [SPARK-13030](https://issues.apache.org/jira/browse/SPARK-13030)). **Note** that +`OneHotEncoderEstimator` will be renamed to `OneHotEncoder` in `3.0` (but +`OneHotEncoderEstimator` will be kept as an alias). + +**Changes of behavior** + +* [SPARK-21027](https://issues.apache.org/jira/browse/SPARK-21027): + The default parallelism used in `OneVsRest` is now set to 1 (i.e. serial). In `2.2` and + earlier versions, the level of parallelism was set to the default threadpool size in Scala. +* [SPARK-22156](https://issues.apache.org/jira/browse/SPARK-22156): + The learning rate update for `Word2Vec` was incorrect when `numIterations` was set greater than + `1`. This will cause training results to be different between `2.3` and earlier versions. +* [SPARK-21681](https://issues.apache.org/jira/browse/SPARK-21681): + Fixed an edge case bug in multinomial logistic regression that resulted in incorrect coefficients + when some features had zero variance. +* [SPARK-16957](https://issues.apache.org/jira/browse/SPARK-16957): + Tree algorithms now use mid-points for split values. This may change results from model training. +* [SPARK-14657](https://issues.apache.org/jira/browse/SPARK-14657): + Fixed an issue where the features generated by `RFormula` without an intercept were inconsistent + with the output in R. This may change results from model training in this scenario. + +## Upgrading from MLlib 2.1 to 2.2 ### Breaking changes +{:.no_toc} There are no breaking changes. ### Deprecations and changes of behavior +{:.no_toc} **Deprecations** @@ -45,9 +109,10 @@ There are no deprecations. `StringIndexer` now handles `NULL` values in the same way as unseen values. Previously an exception would always be thrown regardless of the setting of the `handleInvalid` parameter. -## From 2.0 to 2.1 +## Upgrading from MLlib 2.0 to 2.1 ### Breaking changes +{:.no_toc} **Deprecated methods removed** @@ -59,6 +124,7 @@ There are no deprecations. * `validateParams` in `Evaluator` ### Deprecations and changes of behavior +{:.no_toc} **Deprecations** @@ -74,9 +140,10 @@ There are no deprecations. * [SPARK-17389](https://issues.apache.org/jira/browse/SPARK-17389): `KMeans` reduces the default number of steps from 5 to 2 for the k-means|| initialization mode. -## From 1.6 to 2.0 +## Upgrading from MLlib 1.6 to 2.0 ### Breaking changes +{:.no_toc} There were several breaking changes in Spark 2.0, which are outlined below. @@ -171,6 +238,7 @@ Several deprecated methods were removed in the `spark.mllib` and `spark.ml` pack A full list of breaking changes can be found at [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810). ### Deprecations and changes of behavior +{:.no_toc} **Deprecations** @@ -221,7 +289,7 @@ Changes of behavior in the `spark.mllib` and `spark.ml` packages include: `QuantileDiscretizer` now uses `spark.sql.DataFrameStatFunctions.approxQuantile` to find splits (previously used custom sampling logic). The output buckets will differ for same input data and params. -## From 1.5 to 1.6 +## Upgrading from MLlib 1.5 to 1.6 There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are deprecations and changes of behavior. @@ -248,7 +316,7 @@ Changes of behavior: tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the behavior of the simpler `Tokenizer` transformer. -## From 1.4 to 1.5 +## Upgrading from MLlib 1.4 to 1.5 In the `spark.mllib` package, there are no breaking API changes but several behavior changes: @@ -267,7 +335,7 @@ In the `spark.ml` package, there exists one breaking API change and one behavior * [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. -## From 1.3 to 1.4 +## Upgrading from MLlib 1.3 to 1.4 In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: @@ -286,7 +354,7 @@ Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API changes for future releases. -## From 1.2 to 1.3 +## Upgrading from MLlib 1.2 to 1.3 In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. @@ -313,7 +381,7 @@ Other changes were in `LogisticRegression`: * The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). * In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. -## From 1.1 to 1.2 +## Upgrading from MLlib 1.1 to 1.2 The only API changes in MLlib v1.2 are in [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), @@ -339,7 +407,7 @@ The tree `Node` now includes more information, including the probability of the Examples in the Spark distribution and examples in the [Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. -## From 1.0 to 1.1 +## Upgrading from MLlib 1.0 to 1.1 The only API changes in MLlib v1.1 are in [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), @@ -365,7 +433,7 @@ simple `String` types. Examples of the new recommended `trainClassifier` and `trainRegressor` are given in the [Decision Trees Guide](mllib-decision-tree.html#examples). -## From 0.9 to 1.0 +## Upgrading from MLlib 0.9 to 1.0 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few breaking changes. If your data is sparse, please store it in a sparse format instead of dense to diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md new file mode 100644 index 0000000000000..889941c37bf43 --- /dev/null +++ b/docs/pyspark-migration-guide.md @@ -0,0 +1,120 @@ +--- +layout: global +title: "Migration Guide: PySpark (Python on Spark)" +displayTitle: "Migration Guide: PySpark (Python on Spark)" +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +* Table of contents +{:toc} + +Note that this migration guide describes the items specific to PySpark. +Many items of SQL migration can be applied when migrating PySpark to higher versions. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from PySpark 2.4 to 3.0 + + - Since Spark 3.0, PySpark requires a Pandas version of 0.23.2 or higher to use Pandas related functionality, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + + - Since Spark 3.0, PySpark requires a PyArrow version of 0.12.1 or higher to use PyArrow related functionality, such as `pandas_udf`, `toPandas` and `createDataFrame` with "spark.sql.execution.arrow.enabled=true", etc. + + - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. + + - In PySpark, when Arrow optimization is enabled, if Arrow version is higher than 0.11.0, Arrow can perform safe type conversion when converting Pandas.Series to Arrow array during serialization. Arrow will raise errors when detecting unsafe type conversion like overflow. Setting `spark.sql.execution.pandas.arrowSafeTypeConversion` to true can enable it. The default setting is false. PySpark's behavior for Arrow versions is illustrated in the table below: + + + + + + + + + + + + + + + + + + + + + +
    + PyArrow version + + Integer Overflow + + Floating Point Truncation +
    + version < 0.11.0 + + Raise error + + Silently allows +
    + version > 0.11.0, arrowSafeTypeConversion=false + + Silent overflow + + Silently allows +
    + version > 0.11.0, arrowSafeTypeConversion=true + + Raise error + + Raise error +
    + + - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. + +## Upgrading from PySpark 2.3 to 2.4 + + - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. + +## Upgrading from PySpark 2.3.0 to 2.3.1 and above + + - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. + +## Upgrading from PySpark 2.2 to 2.3 + + - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + + - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. + + - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. + + - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. + +## Upgrading from PySpark 1.4 to 1.5 + + - Resolution of strings to columns in Python now supports using dots (`.`) to qualify the column or + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + + - DataFrame.withColumn method in PySpark supports adding a new column or replacing existing columns of the same name. + + +## Upgrading from PySpark 1.0-1.2 to 1.3 + +#### Python DataTypes No Longer Singletons +{:.no_toc} + +When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of +referencing a singleton. diff --git a/docs/sparkr-migration-guide.md b/docs/sparkr-migration-guide.md new file mode 100644 index 0000000000000..6fbc4c03aefc1 --- /dev/null +++ b/docs/sparkr-migration-guide.md @@ -0,0 +1,77 @@ +--- +layout: global +title: "Migration Guide: SparkR (R on Spark)" +displayTitle: "Migration Guide: SparkR (R on Spark)" +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +* Table of contents +{:toc} + +Note that this migration guide describes the items specific to SparkR. +Many items of SQL migration can be applied when migrating SparkR to higher versions. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from SparkR 2.4 to 3.0 + + - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead. + - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, and `dropTempTable` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead. + +## Upgrading from SparkR 2.3 to 2.4 + + - Previously, we don't check the validity of the size of the last layer in `spark.mlp`. For example, if the training data only has two labels, a `layers` param like `c(1, 3)` doesn't cause an error previously, now it does. + +## Upgrading from SparkR 2.3 to 2.3.1 and above + + - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-based. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. + +## Upgrading from SparkR 2.2 to 2.3 + + - The `stringsAsFactors` parameter was previously ignored with `collect`, for example, in `collect(createDataFrame(iris), stringsAsFactors = TRUE))`. It has been corrected. + - For `summary`, option for statistics to compute has been added. Its output is changed from that from `describe`. + - A warning can be raised if versions of SparkR package and the Spark JVM do not match. + +## Upgrading from SparkR 2.1 to 2.2 + + - A `numPartitions` parameter has been added to `createDataFrame` and `as.DataFrame`. When splitting the data, the partition position calculation has been made to match the one in Scala. + - The method `createExternalTable` has been deprecated to be replaced by `createTable`. Either methods can be called to create external or managed table. Additional catalog methods have also been added. + - By default, derby.log is now saved to `tempdir()`. This will be created when instantiating the SparkSession with `enableHiveSupport` set to `TRUE`. + - `spark.lda` was not setting the optimizer correctly. It has been corrected. + - Several model summary outputs are updated to have `coefficients` as `matrix`. This includes `spark.logit`, `spark.kmeans`, `spark.glm`. Model summary outputs for `spark.gaussianMixture` have added log-likelihood as `loglik`. + +## Upgrading from SparkR 2.0 to 3.1 + + - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. + + +## Upgrading from SparkR 1.6 to 2.0 + + - The method `table` has been removed and replaced by `tableToDF`. + - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. + - Spark's `SQLContext` and `HiveContext` have been deprecated to be replaced by `SparkSession`. Instead of `sparkR.init()`, call `sparkR.session()` in its place to instantiate the SparkSession. Once that is done, that currently active SparkSession will be used for SparkDataFrame operations. + - The parameter `sparkExecutorEnv` is not supported by `sparkR.session`. To set environment for the executors, set Spark config properties with the prefix "spark.executorEnv.VAR_NAME", for example, "spark.executorEnv.PATH" + - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. + - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. + - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. + - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` + +## Upgrading from SparkR 1.5 to 1.6 + + - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. + - SparkSQL converts `NA` in R to `null` and vice-versa. + - Since 1.6.1, withColumn method in SparkR supports adding a new column to or replacing existing columns + of the same name of a DataFrame. diff --git a/docs/sparkr.md b/docs/sparkr.md index 7431d025aa629..c36d1a23d0d2e 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -748,49 +748,5 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma # Migration Guide -## Upgrading From SparkR 1.5.x to 1.6.x +The migration guide is now archived [on this page](sparkr-migration-guide.html). - - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. - - SparkSQL converts `NA` in R to `null` and vice-versa. - -## Upgrading From SparkR 1.6.x to 2.0 - - - The method `table` has been removed and replaced by `tableToDF`. - - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. - - Spark's `SQLContext` and `HiveContext` have been deprecated to be replaced by `SparkSession`. Instead of `sparkR.init()`, call `sparkR.session()` in its place to instantiate the SparkSession. Once that is done, that currently active SparkSession will be used for SparkDataFrame operations. - - The parameter `sparkExecutorEnv` is not supported by `sparkR.session`. To set environment for the executors, set Spark config properties with the prefix "spark.executorEnv.VAR_NAME", for example, "spark.executorEnv.PATH" - - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. - - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. - - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. - - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` - -## Upgrading to SparkR 2.1.0 - - - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. - -## Upgrading to SparkR 2.2.0 - - - A `numPartitions` parameter has been added to `createDataFrame` and `as.DataFrame`. When splitting the data, the partition position calculation has been made to match the one in Scala. - - The method `createExternalTable` has been deprecated to be replaced by `createTable`. Either methods can be called to create external or managed table. Additional catalog methods have also been added. - - By default, derby.log is now saved to `tempdir()`. This will be created when instantiating the SparkSession with `enableHiveSupport` set to `TRUE`. - - `spark.lda` was not setting the optimizer correctly. It has been corrected. - - Several model summary outputs are updated to have `coefficients` as `matrix`. This includes `spark.logit`, `spark.kmeans`, `spark.glm`. Model summary outputs for `spark.gaussianMixture` have added log-likelihood as `loglik`. - -## Upgrading to SparkR 2.3.0 - - - The `stringsAsFactors` parameter was previously ignored with `collect`, for example, in `collect(createDataFrame(iris), stringsAsFactors = TRUE))`. It has been corrected. - - For `summary`, option for statistics to compute has been added. Its output is changed from that from `describe`. - - A warning can be raised if versions of SparkR package and the Spark JVM do not match. - -## Upgrading to SparkR 2.3.1 and above - - - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-based. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. - -## Upgrading to SparkR 2.4.0 - - - Previously, we don't check the validity of the size of the last layer in `spark.mlp`. For example, if the training data only has two labels, a `layers` param like `c(1, 3)` doesn't cause an error previously, now it does. - -## Upgrading to SparkR 3.0.0 - - - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead. - - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, and `dropTempTable` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead. diff --git a/docs/sql-migration-guide-hive-compatibility.md b/docs/sql-migration-guide-hive-compatibility.md deleted file mode 100644 index d4b4fdf19d926..0000000000000 --- a/docs/sql-migration-guide-hive-compatibility.md +++ /dev/null @@ -1,167 +0,0 @@ ---- -layout: global -title: Compatibility with Apache Hive -displayTitle: Compatibility with Apache Hive -license: | - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. ---- - -* Table of contents -{:toc} - -Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. -Currently, Hive SerDes and UDFs are based on Hive 1.2.1, -and Spark SQL can be connected to different versions of Hive Metastore -(from 0.12.0 to 2.3.6 and 3.0.0 to 3.1.2. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). - -#### Deploying in Existing Hive Warehouses - -The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY` - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathematical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDes) -* Window functions -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub-queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables including dynamic partition insertion -* View - * If column aliases are not specified in view definition queries, both Spark and Hive will - generate alias names, but in different ways. In order for Spark to be able to read views created - by Hive, users should explicitly specify column aliases in view definition queries. As an - example, Spark cannot read `v1` created as below by Hive. - - ``` - CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2; - ``` - - Instead, you should create `v1` as below with column aliases explicitly specified. - - ``` - CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2; - ``` - -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `DATE` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - - -**Esoteric Hive Features** - -* `UNION` type -* Unique join -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment and only supports populating the sizeInBytes field of the hive metastore. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -less important due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block-level bitmap indexes and virtual columns (used to build indexes) -* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". -* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -**Hive UDF/UDTF/UDAF** - -Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: - -* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically - include additional resources required by this UDF. -* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses - a deprecated interface `initialize(ObjectInspector[])` only. -* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize - functions with `MapredContext`, which is inapplicable to Spark. -* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. - Spark SQL does not call this function when tasks finish. -* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. - Spark SQL currently does not support the reuse of aggregation. -* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating - an aggregate over a fixed window. - -### Incompatible Hive UDF - -Below are the scenarios in which Hive and Spark generate different results: - -* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. -* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. -* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md deleted file mode 100644 index 5add2fb93937d..0000000000000 --- a/docs/sql-migration-guide-upgrade.md +++ /dev/null @@ -1,834 +0,0 @@ ---- -layout: global -title: Spark SQL Upgrading Guide -displayTitle: Spark SQL Upgrading Guide -license: | - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. ---- - -* Table of contents -{:toc} - -## Upgrading From Spark SQL 2.4 to 3.0 - - - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. - - - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead. - - - Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. - - - Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. - - - Since Spark 3.0, PySpark requires a Pandas version of 0.23.2 or higher to use Pandas related functionality, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - - Since Spark 3.0, PySpark requires a PyArrow version of 0.12.1 or higher to use PyArrow related functionality, such as `pandas_udf`, `toPandas` and `createDataFrame` with "spark.sql.execution.arrow.enabled=true", etc. - - - In Spark version 2.4 and earlier, SQL queries such as `FROM ` or `FROM
    UNION ALL FROM
    ` are supported by accident. In hive-style `FROM
    SELECT `, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. Therefore we will treat these queries as invalid since Spark 3.0. - - - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. - - - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. - - - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType` and `DoubleType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. - - - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. - - - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful. - - - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. - - - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and earlier. - - - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be undefined. - - - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. - - - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - - - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. - - - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. - - - Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and earlier, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. Since Spark 3.0, cache name and storage level will be first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. - - - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. - - - In PySpark, when Arrow optimization is enabled, if Arrow version is higher than 0.11.0, Arrow can perform safe type conversion when converting Pandas.Series to Arrow array during serialization. Arrow will raise errors when detecting unsafe type conversion like overflow. Setting `spark.sql.execution.pandas.arrowSafeTypeConversion` to true can enable it. The default setting is false. PySpark's behavior for Arrow versions is illustrated in the table below: -
    - - - - - - - - - - - - - - - - - - - - -
    - PyArrow version - - Integer Overflow - - Floating Point Truncation -
    - version < 0.11.0 - - Raise error - - Silently allows -
    - version > 0.11.0, arrowSafeTypeConversion=false - - Silent overflow - - Silently allows -
    - version > 0.11.0, arrowSafeTypeConversion=true - - Raise error - - Raise error -
    - - - In Spark version 2.4 and earlier, if `org.apache.spark.sql.functions.udf(Any, DataType)` gets a Scala closure with primitive-type argument, the returned UDF will return null if the input values is null. Since Spark 3.0, the UDF will return the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` will return null in Spark 2.4 and earlier if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default. - - - Since Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and etc. Spark 3.0 uses Java 8 API classes from the java.time packages that based on ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and earlier, those operations are performed by using the hybrid calendar (Julian + Gregorian, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: - - - CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. - - - The `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. - - - The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. - - - the JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. - - - Formatting of `TIMESTAMP` and `DATE` literals. - - - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. - - - In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. - - - In Spark version 2.4 and earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`. - - - Since Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion uses the default time zone of the Java virtual machine. - - - In Spark version 2.4, when a spark session is created via `cloneSession()`, the newly created spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent spark session. Since Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. The old behavior can be restored by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`. - - - Since Spark 3.0, parquet logical type `TIMESTAMP_MICROS` is used by default while saving `TIMESTAMP` columns. In Spark version 2.4 and earlier, `TIMESTAMP` columns are saved as `INT96` in parquet files. To set `INT96` to `spark.sql.parquet.outputTimestampType` restores the previous behavior. - - - Since Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it will fallback to hive-site.xml present in the `Hadoop configuration` of `SparkContext`. - - - Since Spark 3.0, Spark will cast `String` to `Date/TimeStamp` in binary comparisons with dates/timestamps. The previous behaviour of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString` to `true`. - - - Since Spark 3.0, when Avro files are written with user provided schema, the fields will be matched by field names between catalyst schema and avro schema instead of positions. - - - Since Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark will throw runtime NPE if any of the records contains null. - - - Since Spark 3.0, we use a new protocol for fetching shuffle blocks, for external shuffle service users, we need to upgrade the server correspondingly. Otherwise, we'll get the error message `UnsupportedOperationException: Unexpected message: FetchShuffleBlocks`. If it is hard to upgrade the shuffle service right now, you can still use the old protocol by setting `spark.shuffle.useOldFetchProtocol` to `true`. - - - Since Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, i.e., if the `predicate` returns any `null`s and no `true` is obtained, then `exists` will return `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` will be `null`. The previous behaviour can be restored by setting `spark.sql.legacy.arrayExistsFollowsThreeValuedLogic` to `false`. - - - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. - - - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. - - - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. - - - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. - - - Since Spark 3.0, 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and earlier, 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. - - - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`. - - - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. - - - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Operation - - Result prior to Spark 3.0 - - Result starting Spark 3.0 -
    - CAST('infinity' AS DOUBLE)
    - CAST('+infinity' AS DOUBLE)
    - CAST('inf' AS DOUBLE)
    - CAST('+inf' AS DOUBLE)
    -
    - NULL - - Double.PositiveInfinity -
    - CAST('-infinity' AS DOUBLE)
    - CAST('-inf' AS DOUBLE)
    -
    - NULL - - Double.NegativeInfinity -
    - CAST('infinity' AS FLOAT)
    - CAST('+infinity' AS FLOAT)
    - CAST('inf' AS FLOAT)
    - CAST('+inf' AS FLOAT)
    -
    - NULL - - Float.PositiveInfinity -
    - CAST('-infinity' AS FLOAT)
    - CAST('-inf' AS FLOAT)
    -
    - NULL - - Float.NegativeInfinity -
    - CAST('nan' AS DOUBLE) - - NULL - - Double.NaN -
    - CAST('nan' AS FLOAT) - - NULL - - Float.NaN -
    - -## Upgrading from Spark SQL 2.4 to 2.4.1 - - - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was - inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code. - Unitless values are now consistently interpreted as milliseconds. Applications that set values like "30" - need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise, - the extremely short interval that results will likely cause applications to fail. - - - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. - -## Upgrading From Spark SQL 2.3 to 2.4 - - - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Query - - Spark 2.3 or Prior - - Spark 2.4 - - Remarks -
    - SELECT array_contains(array(1), 1.34D); - - true - - false - - In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively. -
    - SELECT array_contains(array(1), '1'); - - true - - AnalysisException is thrown. - - Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. -
    - SELECT array_contains(array(1), 'anystring'); - - null - - AnalysisException is thrown. - - Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. -
    - - - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - - - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - - - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - - - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - - - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use `spark.sql.orc.impl=hive` to create the files shared with Hive 2.1.1 and older. - - - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. - - - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. - - - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - - - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - - - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - - - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - - - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - - - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - - - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - - - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - - - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - - - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - - - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. - - - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - - - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - - - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. - - - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. - - - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. - -## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - - - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. - -## Upgrading From Spark SQL 2.2 to 2.3 - - - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. - - - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. - - - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - - - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - InputA \ InputB - - NullType - - IntegerType - - LongType - - DecimalType(38,0)* - - DoubleType - - DateType - - TimestampType - - StringType -
    - NullType - NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    - IntegerType - IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    - LongType - LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DecimalType(38,0)* - DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DoubleType - DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    - DateType - DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    - TimestampType - TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    - StringType - StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    - - Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. - - - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. - - - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. - - - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). - - - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. - - - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. - - - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes - - - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). - - - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. - - - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. - - - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. - - - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. - - - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. - -## Upgrading From Spark SQL 2.1 to 2.2 - - - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. - - - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). - - - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. - -## Upgrading From Spark SQL 2.0 to 2.1 - - - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - - - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. - - - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. - - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. - - - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. - - - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. - -## Upgrading From Spark SQL 1.6 to 2.0 - - - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and - - `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - - - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for - `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed - transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., - `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages’ - APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the - single-node data frame notion in these languages. - - - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` - - - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` - - - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` - - - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. - - - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` - in order to prevent accidental dropping the existing data in the user-provided locations. - That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. - Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. - Note that this is different from the Hive behavior. - - - As a result, `DROP TABLE` statements on those tables will not remove the data. - - - `spark.sql.parquet.cacheMetadata` is no longer used. - See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. - -## Upgrading From Spark SQL 1.5 to 1.6 - - - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC - connection owns a copy of their own SQL configuration and temporary function registry. Cached - tables are still shared though. If you prefer to run the Thrift server in the old single-session - mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add - this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: - - {% highlight bash %} - ./sbin/start-thriftserver.sh \ - --conf spark.sql.hive.thriftServer.singleSession=true \ - ... - {% endhighlight %} - - - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns - of the same name of a DataFrame. - - - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This - change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType - from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for - details. - -## Upgrading From Spark SQL 1.4 to 1.5 - - - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with - code generation for expression evaluation. These features can both be disabled by setting - `spark.sql.tungsten.enabled` to `false`. - - - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting - `spark.sql.parquet.mergeSchema` to `true`. - - - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or - access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - - - In-memory columnar storage partition pruning is on by default. It can be disabled by setting - `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - - - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum - precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now - used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. - - - Timestamps are now stored at a precision of 1us, rather than 1ns - - - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains - unchanged. - - - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - - - JSON data source will not automatically load new files that are created by other applications - (i.e. files that are not inserted to the dataset through Spark SQL). - For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), - users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method - to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate - the DataFrame and the new DataFrame will include new files. - - - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. - -## Upgrading from Spark SQL 1.3 to 1.4 - -#### DataFrame data reader/writer interface - -Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) -and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). - -See the API docs for `SQLContext.read` ( - Scala, - Java, - Python -) and `DataFrame.write` ( - Scala, - Java, - Python -) more information. - - -#### DataFrame.groupBy retains grouping columns - -Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the -grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. - -
    -
    -{% highlight scala %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg($"department", max("age"), sum("expense")) - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")) - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    -{% highlight java %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(col("department"), max("age"), sum("expense")); - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")); - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false"); - -{% endhighlight %} -
    - -
    -{% highlight python %} - -import pyspark.sql.functions as func - -# In 1.3.x, in order for the grouping column "department" to show up, -# it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) - -# In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(func.max("age"), func.sum("expense")) - -# Revert to 1.3.x behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    - - -#### Behavior change on DataFrame.withColumn - -Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added -as a new column with its specified name in the result DataFrame even if there may be any existing -columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different -name from names of all existing columns or replacing existing columns of the same name. - -Note that this change is only for Scala API, not for PySpark and SparkR. - - -## Upgrading from Spark SQL 1.0-1.2 to 1.3 - -In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the -available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other -releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked -as unstable (i.e., DeveloperAPI or Experimental). - -#### Rename of SchemaRDD to DataFrame - -The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has -been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD -directly, but instead provide most of the functionality that RDDs provide though their own -implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. - -In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for -some use cases. It is still recommended that users update their code to use `DataFrame` instead. -Java and Python users will need to update their code. - -#### Unification of the Java and Scala APIs - -Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) -that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users -of either language should use `SQLContext` and `DataFrame`. In general these classes try to -use types that are usable from both languages (i.e. `Array` instead of language-specific collections). -In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading -is used instead. - -Additionally, the Java specific types API has been removed. Users of both Scala and Java should -use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. - - -#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) - -Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought -all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit -conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. -Users should now write `import sqlContext.implicits._`. - -Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., -case classes or tuples) with a method `toDF`, instead of applying automatically. - -When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import -`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: -`import org.apache.spark.sql.functions._`. - -#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) - -Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users -should instead import the classes in `org.apache.spark.sql.types` - -#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) - -Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been -moved into the udf object in `SQLContext`. - -
    -
    -{% highlight scala %} - -sqlContext.udf.register("strLen", (s: String) => s.length()) - -{% endhighlight %} -
    - -
    -{% highlight java %} - -sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); - -{% endhighlight %} -
    - -
    - -Python UDF registration is unchanged. - -#### Python DataTypes No Longer Singletons - -When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of -referencing a singleton. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 4c23147106b65..71e533eef5fe6 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -1,7 +1,7 @@ --- layout: global -title: Migration Guide -displayTitle: Migration Guide +title: "Migration Guide: SQL, Datasets and DataFrame" +displayTitle: "Migration Guide: SQL, Datasets and DataFrame" license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,20 +19,892 @@ license: | limitations under the License. --- -* [Spark SQL Upgrading Guide](sql-migration-guide-upgrade.html) - * [Upgrading From Spark SQL 2.4 to 3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30) - * [Upgrading From Spark SQL 2.3 to 2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24) - * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above) - * [Upgrading From Spark SQL 2.2 to 2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23) - * [Upgrading From Spark SQL 2.1 to 2.2](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-21-to-22) - * [Upgrading From Spark SQL 2.0 to 2.1](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-20-to-21) - * [Upgrading From Spark SQL 1.6 to 2.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-16-to-20) - * [Upgrading From Spark SQL 1.5 to 1.6](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-15-to-16) - * [Upgrading From Spark SQL 1.4 to 1.5](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-14-to-15) - * [Upgrading from Spark SQL 1.3 to 1.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-13-to-14) - * [Upgrading from Spark SQL 1.0-1.2 to 1.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-10-12-to-13) -* [Compatibility with Apache Hive](sql-migration-guide-hive-compatibility.html) - * [Deploying in Existing Hive Warehouses](sql-migration-guide-hive-compatibility.html#deploying-in-existing-hive-warehouses) - * [Supported Hive Features](sql-migration-guide-hive-compatibility.html#supported-hive-features) - * [Unsupported Hive Functionality](sql-migration-guide-hive-compatibility.html#unsupported-hive-functionality) - * [Incompatible Hive UDF](sql-migration-guide-hive-compatibility.html#incompatible-hive-udf) +* Table of contents +{:toc} + +## Upgrading from Spark SQL 2.4 to 3.0 + + - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. + + - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead. + + - Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. + + - Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. + + - In Spark version 2.4 and earlier, SQL queries such as `FROM ` or `FROM
    UNION ALL FROM
    ` are supported by accident. In hive-style `FROM
    SELECT `, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. Therefore we will treat these queries as invalid since Spark 3.0. + + - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. + + - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType` and `DoubleType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. + + - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. + + - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. + + - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful. + + - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. + + - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and earlier. + + - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be undefined. + + - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. + + - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. + + - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. + + - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. + + - Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and earlier, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. Since Spark 3.0, cache name and storage level will be first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. + + - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. + + - In Spark version 2.4 and earlier, if `org.apache.spark.sql.functions.udf(Any, DataType)` gets a Scala closure with primitive-type argument, the returned UDF will return null if the input values is null. Since Spark 3.0, the UDF will return the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` will return null in Spark 2.4 and earlier if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default. + + - Since Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and etc. Spark 3.0 uses Java 8 API classes from the java.time packages that based on ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and earlier, those operations are performed by using the hybrid calendar (Julian + Gregorian, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: + + - CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. + + - The `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. + + - The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. + + - the JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. + + - Formatting of `TIMESTAMP` and `DATE` literals. + + - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. + + - In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. + + - In Spark version 2.4 and earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`. + + - Since Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion uses the default time zone of the Java virtual machine. + + - In Spark version 2.4, when a spark session is created via `cloneSession()`, the newly created spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent spark session. Since Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. The old behavior can be restored by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`. + + - Since Spark 3.0, parquet logical type `TIMESTAMP_MICROS` is used by default while saving `TIMESTAMP` columns. In Spark version 2.4 and earlier, `TIMESTAMP` columns are saved as `INT96` in parquet files. To set `INT96` to `spark.sql.parquet.outputTimestampType` restores the previous behavior. + + - Since Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it will fallback to hive-site.xml present in the `Hadoop configuration` of `SparkContext`. + + - Since Spark 3.0, Spark will cast `String` to `Date/TimeStamp` in binary comparisons with dates/timestamps. The previous behaviour of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString` to `true`. + + - Since Spark 3.0, when Avro files are written with user provided schema, the fields will be matched by field names between catalyst schema and avro schema instead of positions. + + - Since Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark will throw runtime NPE if any of the records contains null. + + - Since Spark 3.0, we use a new protocol for fetching shuffle blocks, for external shuffle service users, we need to upgrade the server correspondingly. Otherwise, we'll get the error message `UnsupportedOperationException: Unexpected message: FetchShuffleBlocks`. If it is hard to upgrade the shuffle service right now, you can still use the old protocol by setting `spark.shuffle.useOldFetchProtocol` to `true`. + + - Since Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, i.e., if the `predicate` returns any `null`s and no `true` is obtained, then `exists` will return `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` will be `null`. The previous behaviour can be restored by setting `spark.sql.legacy.arrayExistsFollowsThreeValuedLogic` to `false`. + + - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. + + - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. + + - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. + + - Since Spark 3.0, 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and earlier, 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. + + - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`. + + - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. + + - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below: +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Operation + + Result prior to Spark 3.0 + + Result starting Spark 3.0 +
    + CAST('infinity' AS DOUBLE)
    + CAST('+infinity' AS DOUBLE)
    + CAST('inf' AS DOUBLE)
    + CAST('+inf' AS DOUBLE)
    +
    + NULL + + Double.PositiveInfinity +
    + CAST('-infinity' AS DOUBLE)
    + CAST('-inf' AS DOUBLE)
    +
    + NULL + + Double.NegativeInfinity +
    + CAST('infinity' AS FLOAT)
    + CAST('+infinity' AS FLOAT)
    + CAST('inf' AS FLOAT)
    + CAST('+inf' AS FLOAT)
    +
    + NULL + + Float.PositiveInfinity +
    + CAST('-infinity' AS FLOAT)
    + CAST('-inf' AS FLOAT)
    +
    + NULL + + Float.NegativeInfinity +
    + CAST('nan' AS DOUBLE) + + NULL + + Double.NaN +
    + CAST('nan' AS FLOAT) + + NULL + + Float.NaN +
    + +## Upgrading from Spark SQL 2.4 to 2.4.1 + + - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was + inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code. + Unitless values are now consistently interpreted as milliseconds. Applications that set values like "30" + need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise, + the extremely short interval that results will likely cause applications to fail. + + - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. + +## Upgrading from Spark SQL 2.3 to 2.4 + + - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Query + + Spark 2.3 or Prior + + Spark 2.4 + + Remarks +
    + SELECT array_contains(array(1), 1.34D); + + true + + false + + In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively. +
    + SELECT array_contains(array(1), '1'); + + true + + AnalysisException is thrown. + + Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. +
    + SELECT array_contains(array(1), 'anystring'); + + null + + AnalysisException is thrown. + + Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. +
    + + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. + + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. + + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. + + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. + + - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use `spark.sql.orc.impl=hive` to create the files shared with Hive 2.1.1 and older. + + - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. + + - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. + + - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. + + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. + + - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. + + - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. + + - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. + + - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. + + - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. + + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. + + - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. + + - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. + +## Upgrading from Spark SQL 2.2 to 2.3 + + - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + + - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. + + - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + InputA \ InputB + + NullType + + IntegerType + + LongType + + DecimalType(38,0)* + + DoubleType + + DateType + + TimestampType + + StringType +
    + NullType + NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    + IntegerType + IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    + LongType + LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DecimalType(38,0)* + DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DoubleType + DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    + DateType + DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    + TimestampType + TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    + StringType + StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    + + Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. + + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + + - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. + + - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes + + - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). + + - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. + + - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. + + - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. + + - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. + +## Upgrading from Spark SQL 2.1 to 2.2 + + - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. + + - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). + + - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. + +## Upgrading from Spark SQL 2.0 to 2.1 + + - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. + - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. + +## Upgrading from Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the + single-node data frame notion in these languages. + + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + + - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` + in order to prevent accidental dropping the existing data in the user-provided locations. + That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. + Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. + Note that this is different from the Hive behavior. + + - As a result, `DROP TABLE` statements on those tables will not remove the data. + + - `spark.sql.parquet.cacheMetadata` is no longer used. + See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. + +## Upgrading from Spark SQL 1.5 to 1.6 + + - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC + connection owns a copy of their own SQL configuration and temporary function registry. Cached + tables are still shared though. If you prefer to run the Thrift server in the old single-session + mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add + this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: + + {% highlight bash %} + ./sbin/start-thriftserver.sh \ + --conf spark.sql.hive.thriftServer.singleSession=true \ + ... + {% endhighlight %} + + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + +## Upgrading from Spark SQL 1.4 to 1.5 + + - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with + code generation for expression evaluation. These features can both be disabled by setting + `spark.sql.tungsten.enabled` to `false`. + + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + `spark.sql.parquet.mergeSchema` to `true`. + + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting + `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + + - Timestamps are now stored at a precision of 1us, rather than 1ns + + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + unchanged. + + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). + + - JSON data source will not automatically load new files that are created by other applications + (i.e. files that are not inserted to the dataset through Spark SQL). + For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), + users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method + to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate + the DataFrame and the new DataFrame will include new files. + +## Upgrading from Spark SQL 1.3 to 1.4 + +#### DataFrame data reader/writer interface +{:.no_toc} + +Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) +and writing data out (`DataFrame.write`), +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). + +See the API docs for `SQLContext.read` ( + Scala, + Java, + Python +) and `DataFrame.write` ( + Scala, + Java, + Python +) more information. + + +#### DataFrame.groupBy retains grouping columns +{:.no_toc} + +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the +grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. + +
    +
    +{% highlight scala %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg($"department", max("age"), sum("expense")) + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")) + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(col("department"), max("age"), sum("expense")); + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")); + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false"); + +{% endhighlight %} +
    + +
    +{% highlight python %} + +import pyspark.sql.functions as func + +# In 1.3.x, in order for the grouping column "department" to show up, +# it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) + +# In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(func.max("age"), func.sum("expense")) + +# Revert to 1.3.x behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    + + +#### Behavior change on DataFrame.withColumn +{:.no_toc} + +Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added +as a new column with its specified name in the result DataFrame even if there may be any existing +columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different +name from names of all existing columns or replacing existing columns of the same name. + +Note that this change is only for Scala API, not for PySpark and SparkR. + + +## Upgrading from Spark SQL 1.0-1.2 to 1.3 + +In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +as unstable (i.e., DeveloperAPI or Experimental). + +#### Rename of SchemaRDD to DataFrame +{:.no_toc} + +The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +directly, but instead provide most of the functionality that RDDs provide though their own +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. + +In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for +some use cases. It is still recommended that users update their code to use `DataFrame` instead. +Java and Python users will need to update their code. + +#### Unification of the Java and Scala APIs +{:.no_toc} + +Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general these classes try to +use types that are usable from both languages (i.e. `Array` instead of language-specific collections). +In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading +is used instead. + +Additionally, the Java specific types API has been removed. Users of both Scala and Java should +use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. + + +#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) +{:.no_toc} + +Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. +Users should now write `import sqlContext.implicits._`. + +Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., +case classes or tuples) with a method `toDF`, instead of applying automatically. + +When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`import org.apache.spark.sql.functions._`. + +#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) +{:.no_toc} + +Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users +should instead import the classes in `org.apache.spark.sql.types` + +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) +{:.no_toc} + +Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been +moved into the udf object in `SQLContext`. + +
    +
    +{% highlight scala %} + +sqlContext.udf.register("strLen", (s: String) => s.length()) + +{% endhighlight %} +
    + +
    +{% highlight java %} + +sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); + +{% endhighlight %} +
    + +
    + +Python UDF registration is unchanged. + + + +## Compatibility with Apache Hive + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. +Currently, Hive SerDes and UDFs are based on Hive 1.2.1, +and Spark SQL can be connected to different versions of Hive Metastore +(from 0.12.0 to 2.3.6 and 3.0.0 to 3.1.2. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). + +#### Deploying in Existing Hive Warehouses +{:.no_toc} + +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +### Supported Hive Features +{:.no_toc} + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY` + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathematical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDes) +* Window functions +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub-queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables including dynamic partition insertion +* View + * If column aliases are not specified in view definition queries, both Spark and Hive will + generate alias names, but in different ways. In order for Spark to be able to read views created + by Hive, users should explicitly specify column aliases in view definition queries. As an + example, Spark cannot read `v1` created as below by Hive. + + ``` + CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2; + ``` + + Instead, you should create `v1` as below with column aliases explicitly specified. + + ``` + CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2; + ``` + +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `DATE` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +### Unsupported Hive Functionality +{:.no_toc} + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + + +**Esoteric Hive Features** + +* `UNION` type +* Unique join +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment and only supports populating the sizeInBytes field of the hive metastore. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +less important due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block-level bitmap indexes and virtual columns (used to build indexes) +* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". +* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +**Hive UDF/UDTF/UDAF** + +Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: + +* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically + include additional resources required by this UDF. +* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses + a deprecated interface `initialize(ObjectInspector[])` only. +* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize + functions with `MapredContext`, which is inapplicable to Spark. +* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. + Spark SQL does not call this function when tasks finish. +* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. + Spark SQL currently does not support the reuse of aggregation. +* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating + an aggregate over a fixed window. + +### Incompatible Hive UDF +{:.no_toc} + +Below are the scenarios in which Hive and Spark generate different results: + +* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. +* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. +* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/mllib-migration-guides.md b/docs/sql-migration-old.md similarity index 73% rename from docs/mllib-migration-guides.md rename to docs/sql-migration-old.md index b746b96e19f07..e100820f6d664 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/sql-migration-old.md @@ -1,7 +1,7 @@ --- layout: global -title: Old Migration Guides - MLlib -displayTitle: Old Migration Guides - MLlib +title: Migration Guide +displayTitle: Migration Guide license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,6 +19,5 @@ license: | limitations under the License. --- -The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). +The migration guide is now archived [on this page](sql-migration-guide.html). -Past migration guides are now stored at [ml-migration-guides.html](ml-migration-guides.html). diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md new file mode 100644 index 0000000000000..b0fd8a8325dff --- /dev/null +++ b/docs/ss-migration-guide.md @@ -0,0 +1,32 @@ +--- +layout: global +title: "Migration Guide: Structured Streaming" +displayTitle: "Migration Guide: Structured Streaming" +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + +* Table of contents +{:toc} + +Note that this migration guide describes the items specific to Structured Streaming. +Many items of SQL migration can be applied when migrating Structured Streaming to higher versions. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from Structured Streaming 2.4 to 3.0 + +- In Spark 3.0, Structured Streaming forces the source schema into nullable when file-based datasources such as text, json, csv, parquet and orc are used via `spark.readStream(...)`. Previously, it respected the nullability in source schema; however, it caused issues tricky to debug with NPE. To restore the previous behavior, set `spark.sql.streaming.fileSource.schema.forceNullable` to `false`. + From 1b99d0cca4b4fb6d193091f92c46c916b70cd84e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 16 Sep 2019 09:46:34 +0900 Subject: [PATCH 06/87] [SPARK-29069][SQL] ResolveInsertInto should not do table lookup ### What changes were proposed in this pull request? It's more clear to only do table lookup in `ResolveTables` rule (for v2 tables) and `ResolveRelations` rule (for v1 tables). `ResolveInsertInto` should only resolve the `InsertIntoStatement` with resolved relations. ### Why are the changes needed? to make the code simpler ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #25774 from cloud-fan/simplify. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/Analyzer.scala | 58 +++++++++---------- 1 file changed, 27 insertions(+), 31 deletions(-) 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 8e6be32bcf70e..413c75d2581d1 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 @@ -671,6 +671,15 @@ class Analyzer( case scala.Right(tableOpt) => tableOpt } v2TableOpt.map(DataSourceV2Relation.create).getOrElse(u) + + case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => + val v2TableOpt = lookupV2Relation(u.multipartIdentifier) match { + case scala.Left((_, _, tableOpt)) => tableOpt + case scala.Right(tableOpt) => tableOpt + } + v2TableOpt.map(DataSourceV2Relation.create).map { v2Relation => + i.copy(table = v2Relation) + }.getOrElse(i) } } @@ -785,41 +794,28 @@ class Analyzer( object ResolveInsertInto extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => - lookupV2Relation(u.multipartIdentifier) match { - case scala.Left((_, _, Some(v2Table: Table))) => - resolveV2Insert(i, v2Table) - case scala.Right(Some(v2Table: Table)) => - resolveV2Insert(i, v2Table) - case _ => - i + case i @ InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) if i.query.resolved => + // ifPartitionNotExists is append with validation, but validation is not supported + if (i.ifPartitionNotExists) { + throw new AnalysisException( + s"Cannot write, IF NOT EXISTS is not supported for table: ${r.table.name}") } - } - - private def resolveV2Insert(i: InsertIntoStatement, table: Table): LogicalPlan = { - val relation = DataSourceV2Relation.create(table) - // ifPartitionNotExists is append with validation, but validation is not supported - if (i.ifPartitionNotExists) { - throw new AnalysisException( - s"Cannot write, IF NOT EXISTS is not supported for table: ${relation.table.name}") - } - val partCols = partitionColumnNames(relation.table) - validatePartitionSpec(partCols, i.partitionSpec) + val partCols = partitionColumnNames(r.table) + validatePartitionSpec(partCols, i.partitionSpec) - val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get) - val query = addStaticPartitionColumns(relation, i.query, staticPartitions) - val dynamicPartitionOverwrite = partCols.size > staticPartitions.size && - conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC + val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get) + val query = addStaticPartitionColumns(r, i.query, staticPartitions) + val dynamicPartitionOverwrite = partCols.size > staticPartitions.size && + conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC - if (!i.overwrite) { - AppendData.byPosition(relation, query) - } else if (dynamicPartitionOverwrite) { - OverwritePartitionsDynamic.byPosition(relation, query) - } else { - OverwriteByExpression.byPosition( - relation, query, staticDeleteExpression(relation, staticPartitions)) - } + if (!i.overwrite) { + AppendData.byPosition(r, query) + } else if (dynamicPartitionOverwrite) { + OverwritePartitionsDynamic.byPosition(r, query) + } else { + OverwriteByExpression.byPosition(r, query, staticDeleteExpression(r, staticPartitions)) + } } private def partitionColumnNames(table: Table): Seq[String] = { From 471a3eff514480cfcbda79bde9294408cc8eb125 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 16 Sep 2019 00:13:07 -0700 Subject: [PATCH 07/87] [SPARK-28932][BUILD][FOLLOWUP] Switch to scala-library compile dependency for JDK11 ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/25638 to switch `scala-library` from `test` dependency to `compile` dependency in `network-common` module. ### Why are the changes needed? Previously, we added `scala-library` as a test dependency to resolve the followings, but it was insufficient to resolve. This PR aims to switch it to compile dependency. ``` $ java -version openjdk version "11.0.3" 2019-04-16 OpenJDK Runtime Environment AdoptOpenJDK (build 11.0.3+7) OpenJDK 64-Bit Server VM AdoptOpenJDK (build 11.0.3+7, mixed mode) $ mvn clean install -pl common/network-common -DskipTests ... [INFO] --- scala-maven-plugin:4.2.0:doc-jar (attach-scaladocs) spark-network-common_2.12 --- error: fatal error: object scala in compiler mirror not found. one error found [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually, run the following on JDK11. ``` $ mvn clean install -pl common/network-common -DskipTests ``` Closes #25800 from dongjoon-hyun/SPARK-28932-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- common/network-common/pom.xml | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index c107af9ceb415..2ee17800c10e4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -35,6 +35,12 @@ + + + org.scala-lang + scala-library + + io.netty @@ -87,13 +93,6 @@ - - - org.scala-lang - scala-library - ${scala.version} - test - log4j log4j From 6297287dfa6e9d30141728c931ed58c8c4966851 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 16 Sep 2019 21:48:07 +0800 Subject: [PATCH 08/87] [SPARK-29061][SQL] Prints bytecode statistics in debugCodegen ### What changes were proposed in this pull request? This pr proposes to print bytecode statistics (max class bytecode size, max method bytecode size, max constant pool size, and # of inner classes) for generated classes in debug prints, `debugCodegen`. Since these metrics are critical for codegen framework developments, I think its worth printing there. This pr intends to enable `debugCodegen` to print these metrics as following; ``` scala> sql("SELECT sum(v) FROM VALUES(1) t(v)").debugCodegen Found 2 WholeStageCodegen subtrees. == Subtree 1 / 2 (maxClassCodeSize:2693; maxMethodCodeSize:124; maxConstantPoolSize:130(0.20% used); numInnerClasses:0) == ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ *(1) HashAggregate(keys=[], functions=[partial_sum(cast(v#0 as bigint))], output=[sum#5L]) +- *(1) LocalTableScan [v#0] Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage1(references); /* 003 */ } ... ``` ### Why are the changes needed? For efficient developments ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested Closes #25766 from maropu/PrintBytecodeStats. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../expressions/codegen/CodeGenerator.scala | 56 ++++++++++++------- .../spark/sql/execution/QueryExecution.scala | 3 +- .../sql/execution/WholeStageCodegenExec.scala | 6 +- .../spark/sql/execution/debug/package.scala | 30 +++++++--- .../execution/WholeStageCodegenSuite.scala | 6 +- .../sql/execution/debug/DebuggingSuite.scala | 45 ++++++++++++++- .../internal/ExecutorSideSQLConfSuite.scala | 2 +- 7 files changed, 112 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 660a1dbaf0aa1..89277706233d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1211,6 +1211,15 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } } +/** + * Java bytecode statistics of a compiled class by Janino. + */ +case class ByteCodeStats(maxMethodCodeSize: Int, maxConstPoolSize: Int, numInnerClasses: Int) + +object ByteCodeStats { + val UNAVAILABLE = ByteCodeStats(-1, -1, -1) +} + object CodeGenerator extends Logging { // This is the default value of HugeMethodLimit in the OpenJDK HotSpot JVM, @@ -1220,6 +1229,9 @@ object CodeGenerator extends Logging { // The max valid length of method parameters in JVM. final val MAX_JVM_METHOD_PARAMS_LENGTH = 255 + // The max number of constant pool entries in JVM. + final val MAX_JVM_CONSTANT_POOL_SIZE = 65535 + // This is the threshold over which the methods in an inner class are grouped in a single // method which is going to be called by the outer class instead of the many small ones final val MERGE_SPLIT_METHODS_THRESHOLD = 3 @@ -1242,9 +1254,9 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. * - * @return a pair of a generated class and the max bytecode size of generated functions. + * @return a pair of a generated class and the bytecode statistics of generated functions. */ - def compile(code: CodeAndComment): (GeneratedClass, Int) = try { + def compile(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = try { cache.get(code) } catch { // Cache.get() may wrap the original exception. See the following URL @@ -1257,7 +1269,7 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. */ - private[this] def doCompile(code: CodeAndComment): (GeneratedClass, Int) = { + private[this] def doCompile(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = { val evaluator = new ClassBodyEvaluator() // A special classloader used to wrap the actual parent classloader of @@ -1296,7 +1308,7 @@ object CodeGenerator extends Logging { s"\n${CodeFormatter.format(code)}" }) - val maxCodeSize = try { + val codeStats = try { evaluator.cook("generated.java", code.body) updateAndGetCompilationStats(evaluator) } catch { @@ -1314,14 +1326,15 @@ object CodeGenerator extends Logging { throw new CompileException(msg, e.getLocation) } - (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) + (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], codeStats) } /** - * Returns the max bytecode size of the generated functions by inspecting janino private fields. + * Returns the bytecode statistics (max method bytecode size, max constant pool size, and + * # of inner classes) of generated classes by inspecting Janino classes. * Also, this method updates the metrics information. */ - private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Int = { + private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): ByteCodeStats = { // First retrieve the generated classes. val classes = { val resultField = classOf[SimpleCompiler].getDeclaredField("result") @@ -1336,11 +1349,13 @@ object CodeGenerator extends Logging { val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") val codeAttrField = codeAttr.getDeclaredField("code") codeAttrField.setAccessible(true) - val codeSizes = classes.flatMap { case (_, classBytes) => - CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) + val codeStats = classes.map { case (_, classBytes) => + val classCodeSize = classBytes.length + CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classCodeSize) try { val cf = new ClassFile(new ByteArrayInputStream(classBytes)) - val stats = cf.methodInfos.asScala.flatMap { method => + val constPoolSize = cf.getConstantPoolSize + val methodCodeSizes = cf.methodInfos.asScala.flatMap { method => method.getAttributes().filter(_.getClass eq codeAttr).map { a => val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) @@ -1353,19 +1368,20 @@ object CodeGenerator extends Logging { byteCodeSize } } - Some(stats) + (methodCodeSizes.max, constPoolSize) } catch { case NonFatal(e) => logWarning("Error calculating stats of compiled class.", e) - None + (-1, -1) } - }.flatten - - if (codeSizes.nonEmpty) { - codeSizes.max - } else { - 0 } + + val (maxMethodSizes, constPoolSize) = codeStats.unzip + ByteCodeStats( + maxMethodCodeSize = maxMethodSizes.max, + maxConstPoolSize = constPoolSize.max, + // Minus 2 for `GeneratedClass` and an outer-most generated class + numInnerClasses = classes.size - 2) } /** @@ -1380,8 +1396,8 @@ object CodeGenerator extends Logging { private val cache = CacheBuilder.newBuilder() .maximumSize(SQLConf.get.codegenCacheMaxEntries) .build( - new CacheLoader[CodeAndComment, (GeneratedClass, Int)]() { - override def load(code: CodeAndComment): (GeneratedClass, Int) = { + new CacheLoader[CodeAndComment, (GeneratedClass, ByteCodeStats)]() { + override def load(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() 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 630d062d6577e..f294a56c60e26 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 @@ -25,6 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule @@ -213,7 +214,7 @@ class QueryExecution( * * @return Sequence of WholeStageCodegen subtrees and corresponding codegen */ - def codegenToSeq(): Seq[(String, String)] = { + def codegenToSeq(): Seq[(String, String, ByteCodeStats)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index ce9a6ea319d5f..f723fcfac6d06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -688,7 +688,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def doExecute(): RDD[InternalRow] = { val (ctx, cleanedSource) = doCodeGen() // try to compile and fallback if it failed - val (_, maxCodeSize) = try { + val (_, compiledCodeStats) = try { CodeGenerator.compile(cleanedSource) } catch { case NonFatal(_) if !Utils.isTesting && sqlContext.conf.codegenFallback => @@ -698,9 +698,9 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) } // Check if compiled code has a too large function - if (maxCodeSize > sqlContext.conf.hugeMethodLimit) { + if (compiledCodeStats.maxMethodCodeSize > sqlContext.conf.hugeMethodLimit) { logInfo(s"Found too long generated codes and JIT optimization might not work: " + - s"the bytecode size ($maxCodeSize) is above the limit " + + s"the bytecode size (${compiledCodeStats.maxMethodCodeSize}) is above the limit " + s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " + s"for this plan (id=$codegenStageId). To avoid this, you can raise the limit " + s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 03adeaaa66569..6a57ef2cafe23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import java.util.Collections import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging @@ -27,7 +28,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeFormatter, CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat @@ -81,11 +82,20 @@ package object debug { def writeCodegen(append: String => Unit, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) append(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") - for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - append(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") + for (((subtree, code, codeStats), i) <- codegenSeq.zipWithIndex) { + val usedConstPoolRatio = if (codeStats.maxConstPoolSize > 0) { + val rt = 100.0 * codeStats.maxConstPoolSize / CodeGenerator.MAX_JVM_CONSTANT_POOL_SIZE + "(%.2f%% used)".format(rt) + } else { + "" + } + val codeStatsStr = s"maxMethodCodeSize:${codeStats.maxMethodCodeSize}; " + + s"maxConstantPoolSize:${codeStats.maxConstPoolSize}$usedConstPoolRatio; " + + s"numInnerClasses:${codeStats.numInnerClasses}" + append(s"== Subtree ${i + 1} / ${codegenSeq.size} ($codeStatsStr) ==\n") append(subtree) append("\nGenerated code:\n") - append(s"${code}\n") + append(s"$code\n") } } @@ -95,7 +105,7 @@ package object debug { * @param plan the query plan for codegen * @return Sequence of WholeStageCodegen subtrees and corresponding codegen */ - def codegenStringSeq(plan: SparkPlan): Seq[(String, String)] = { + def codegenStringSeq(plan: SparkPlan): Seq[(String, String, ByteCodeStats)] = { val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() plan transform { case s: WholeStageCodegenExec => @@ -105,7 +115,13 @@ package object debug { } codegenSubtrees.toSeq.map { subtree => val (_, source) = subtree.doCodeGen() - (subtree.toString, CodeFormatter.format(source)) + val codeStats = try { + CodeGenerator.compile(source)._2 + } catch { + case NonFatal(_) => + ByteCodeStats.UNAVAILABLE + } + (subtree.toString, CodeFormatter.format(source), codeStats) } } @@ -130,7 +146,7 @@ package object debug { * @param query the streaming query for codegen * @return Sequence of WholeStageCodegen subtrees and corresponding codegen */ - def codegenStringSeq(query: StreamingQuery): Seq[(String, String)] = { + def codegenStringSeq(query: StreamingQuery): Seq[(String, String, ByteCodeStats)] = { val w = asStreamExecution(query) if (w.lastExecution != null) { codegenStringSeq(w.lastExecution.executedPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index d8727d5b584f1..6dfb6c85dc9e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeGenerator} +import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec @@ -213,10 +213,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { ignore("SPARK-21871 check if we can get large code size when compiling too long functions") { val codeWithShortFunctions = genGroupByCode(3) - val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) + val (_, ByteCodeStats(maxCodeSize1, _, _)) = CodeGenerator.compile(codeWithShortFunctions) assert(maxCodeSize1 < SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) val codeWithLongFunctions = genGroupByCode(50) - val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions) + val (_, ByteCodeStats(maxCodeSize2, _, _)) = CodeGenerator.compile(codeWithLongFunctions) assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 7a8da7e7669a4..9a48c1ea0f318 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -19,9 +19,15 @@ package org.apache.spark.sql.execution.debug import java.io.ByteArrayOutputStream +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.expressions.codegen.CodegenContext +import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.StructType class DebuggingSuite extends SharedSparkSession { @@ -46,7 +52,7 @@ class DebuggingSuite extends SharedSparkSession { val res = codegenStringSeq(spark.range(10).groupBy(col("id") * 2).count() .queryExecution.executedPlan) assert(res.length == 2) - assert(res.forall{ case (subtree, code) => + assert(res.forall{ case (subtree, code, _) => subtree.contains("Range") && code.contains("Object[]")}) } @@ -90,4 +96,41 @@ class DebuggingSuite extends SharedSparkSession { | id LongType: {} |""".stripMargin)) } + + case class DummyCodeGeneratorPlan(useInnerClass: Boolean) + extends CodegenSupport with LeafExecNode { + override def output: Seq[Attribute] = StructType.fromDDL("d int").toAttributes + override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(spark.sparkContext.emptyRDD[InternalRow]) + override protected def doExecute(): RDD[InternalRow] = sys.error("Not used") + override protected def doProduce(ctx: CodegenContext): String = { + if (useInnerClass) { + val innerClassName = ctx.freshName("innerClass") + ctx.addInnerClass( + s""" + |public class $innerClassName { + | public $innerClassName() {} + |} + """.stripMargin) + } + "" + } + } + + test("Prints bytecode statistics in debugCodegen") { + Seq(true, false).foreach { useInnerClass => + val plan = WholeStageCodegenExec(DummyCodeGeneratorPlan(useInnerClass))(codegenStageId = 0) + + val genCodes = codegenStringSeq(plan) + assert(genCodes.length == 1) + val (_, _, codeStats) = genCodes.head + val expectedNumInnerClasses = if (useInnerClass) 1 else 0 + assert(codeStats.maxMethodCodeSize > 0 && codeStats.maxConstPoolSize > 0 && + codeStats.numInnerClasses == expectedNumInnerClasses) + + val debugCodegenStr = codegenString(plan) + assert(debugCodegenStr.contains("maxMethodCodeSize:")) + assert(debugCodegenStr.contains("maxConstantPoolSize:")) + assert(debugCodegenStr.contains("numInnerClasses:")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 94b73ec186379..c0238069afcc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -100,7 +100,7 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { val res = codegenStringSeq(spark.range(10).groupBy(col("id") * 2).count() .queryExecution.executedPlan) assert(res.length == 2) - assert(res.forall { case (_, code) => + assert(res.forall { case (_, code, _) => (code.contains("* Codegend pipeline") == flag) && (code.contains("// input[") == flag) }) From 67751e26940a16ab6f9950ae66a46b7cb901c102 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Sep 2019 09:08:25 -0500 Subject: [PATCH 09/87] [SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter ### What changes were proposed in this pull request? The previous refactors of the shuffle writers using the shuffle writer plugin resulted in shuffle write metric updates - particularly write times - being lost in particular situations. This patch restores the lost metric updates. ### Why are the changes needed? This fixes a regression. I'm pretty sure that without this, the Spark UI will lose shuffle write time information. ### Does this PR introduce any user-facing change? No change from Spark 2.4. Without this, there would be a user-facing bug in Spark 3.0. ### How was this patch tested? Existing unit tests. Closes #25780 from mccheah/fix-write-metrics. Authored-by: mcheah Signed-off-by: Imran Rashid --- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 2 ++ .../shuffle/ShufflePartitionPairsWriter.scala | 15 ++++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) 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 f59bddc993639..4d11abd36985e 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 @@ -57,6 +57,7 @@ import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; @@ -382,6 +383,7 @@ private void mergeSpillsWithFileStream( ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); OutputStream partitionOutput = writer.openStream(); try { + partitionOutput = new TimeTrackingOutputStream(writeMetrics, partitionOutput); partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); if (compressionCodec != null) { partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala index a988c5e126a76..e0affb858c359 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala @@ -21,7 +21,7 @@ import java.io.{Closeable, IOException, OutputStream} import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.api.ShufflePartitionWriter -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.{BlockId, TimeTrackingOutputStream} import org.apache.spark.util.Utils import org.apache.spark.util.collection.PairsWriter @@ -39,6 +39,7 @@ private[spark] class ShufflePartitionPairsWriter( private var isClosed = false private var partitionStream: OutputStream = _ + private var timeTrackingStream: OutputStream = _ private var wrappedStream: OutputStream = _ private var objOut: SerializationStream = _ private var numRecordsWritten = 0 @@ -59,7 +60,8 @@ private[spark] class ShufflePartitionPairsWriter( private def open(): Unit = { try { partitionStream = partitionWriter.openStream - wrappedStream = serializerManager.wrapStream(blockId, partitionStream) + timeTrackingStream = new TimeTrackingOutputStream(writeMetrics, partitionStream) + wrappedStream = serializerManager.wrapStream(blockId, timeTrackingStream) objOut = serializerInstance.serializeStream(wrappedStream) } catch { case e: Exception => @@ -78,6 +80,7 @@ private[spark] class ShufflePartitionPairsWriter( // Setting these to null will prevent the underlying streams from being closed twice // just in case any stream's close() implementation is not idempotent. wrappedStream = null + timeTrackingStream = null partitionStream = null } { // Normally closing objOut would close the inner streams as well, but just in case there @@ -86,9 +89,15 @@ private[spark] class ShufflePartitionPairsWriter( wrappedStream = closeIfNonNull(wrappedStream) // Same as above - if wrappedStream closes then assume it closes underlying // partitionStream and don't close again in the finally + timeTrackingStream = null partitionStream = null } { - partitionStream = closeIfNonNull(partitionStream) + Utils.tryWithSafeFinally { + timeTrackingStream = closeIfNonNull(timeTrackingStream) + partitionStream = null + } { + partitionStream = closeIfNonNull(partitionStream) + } } } updateBytesWritten() From 5881871ca5156ef0e83c9503d5eac288320147c3 Mon Sep 17 00:00:00 2001 From: hongdd Date: Mon, 16 Sep 2019 11:07:50 -0700 Subject: [PATCH 10/87] [SPARK-26929][SQL] fix table owner use user instead of principal when create table through spark-sql or beeline MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …create table through spark-sql or beeline ## What changes were proposed in this pull request? fix table owner use user instead of principal when create table through spark-sql private val userName = conf.getUser will get ugi's userName which is principal info, and i copy the source code into HiveClientImpl, and use ugi.getShortUserName() instead of ugi.getUserName(). The owner display correctly. ## How was this patch tested? 1. create a table in kerberos cluster 2. use "desc formatted tbName" check owner Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23952 from hddong/SPARK-26929-fix-table-owner. Lead-authored-by: hongdd Co-authored-by: hongdongdong Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 5b2eeb2cf34c0..ae9eca823d00c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging @@ -221,7 +222,7 @@ private[hive] class HiveClientImpl( hiveConf } - private val userName = conf.getUser + private val userName = UserGroupInformation.getCurrentUser.getShortUserName override def getConf(key: String, defaultValue: String): String = { conf.get(key, defaultValue) From 88c8d5eed2bf26ec4cc6ef68d9bdabbcb7ba1b83 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 16 Sep 2019 15:22:04 -0500 Subject: [PATCH 11/87] [SPARK-23539][SS][FOLLOWUP][TESTS] Add UT to ensure existing query doesn't break with default conf of includeHeaders ### What changes were proposed in this pull request? This patch adds new UT to ensure existing query (before Spark 3.0.0) with checkpoint doesn't break with default configuration of "includeHeaders" being introduced via SPARK-23539. This patch also modifies existing test which checks type of columns to also check headers column as well. ### Why are the changes needed? The patch adds missing tests which guarantees backward compatibility of the change of SPARK-23539. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? UT passed. Closes #25792 from HeartSaVioR/SPARK-23539-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Sean Owen --- .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 3 + .../sources/0/0 | Bin 0 -> 93 bytes .../state/0/0/1.delta | Bin 0 -> 182 bytes .../state/0/1/1.delta | Bin 0 -> 160 bytes .../state/0/2/1.delta | Bin 0 -> 256 bytes .../state/0/3/1.delta | Bin 0 -> 244 bytes .../state/0/4/1.delta | Bin 0 -> 240 bytes .../kafka010/KafkaMicroBatchSourceSuite.scala | 80 +++++++++++++++++- 10 files changed, 84 insertions(+), 2 deletions(-) create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/sources/0/0 create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/0/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/1/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/2/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/3/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/4/1.delta diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata new file mode 100644 index 0000000000000..f1b5ab7aa17f0 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata @@ -0,0 +1 @@ +{"id":"fc415a71-f0a2-4c3c-aeaf-f9e258c3f726"} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 new file mode 100644 index 0000000000000..5dbadea57acbe --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1568508285207,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +{"spark-test-topic-2b8619f5-d3c4-4c2d-b5d1-8d9d9458aa62":{"2":3,"4":3,"1":3,"3":3,"0":3}} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/sources/0/0 b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/sources/0/0 new file mode 100644 index 0000000000000000000000000000000000000000..8cf9f8e009ce87d894533fb440d991ffcba255a3 GIT binary patch literal 93 zcmW;6OA3G>5CG8D9b?uFIR4OXqfSAzNK)E}cTaqaw|ulVC^@`0)}vIcPree$=q&lu hAgiQNC&?HM6FT344Q)*g(%|uI6Ig`w!oD7&ZU^ literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/0/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/0/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..5815bbdcc2467001ef9551fd51e414e2f8abc282 GIT binary patch literal 182 zcmeZ?GI7euPtI2DVPIe|VPs%%e&oZF1Y|dGFfcMq+7SHXRL>+<24;Q+ekKOCuhynO zDF+~C0^$lFE-pwc%GNDOEiTb5$uG!E)-_79Ff+7FGu2HoPBzgsNj6H+O)^a})U`;l zOtCaEwMa}fGXm-}G)QJp6=39MU}a)poxz}=5kG^0o5hBKe+Gk@s9_5O&kTkDhW|j| M19hM%P!k*g086JS#{d8T literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/1/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/1/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..e1a065b2b1c78c8d842a402fbe6933db404c55a6 GIT binary patch literal 160 zcmeZ?GI7euPtH~?V_;yI1H@{)x-3aRx`Bg%kztZY-;Yy0lUNy8_!)Sa7=)RG7=lfK zk`4k4AM`4KjN*dCqHNug)Z!A|lKg_qWL={q3o}E@G*jIa<75+ElVqb5-6Yc#LtTp$ q%M?o!Q;WnzGb5lrBUT0`CI%$|Mh*rRCWZip|3Kgab&4oZ9u5GT{Ui7Q literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/2/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/2/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..cce14294e00443261aa2c3a9cf70ba51fd0d9478 GIT binary patch literal 256 zcmeZ?GI7euPtI1o#=yXk!o+<21b4cUM2=%CLx9o z)}}yN2Owq!;tC)xE=VlO)-6dbF3~N?FUU;RHA=EDGqg-I)lD%@HqkXnHcHV=GEFhm zwMel{u{1HYNK7;{0_xK>WMg1tVvrXoVPIgL!Ju@bYzBiMi#7v469YTT3v^lNQD4cg_vO~Sb<@~lW!*&+4Mmwn1Ry#Cm8kE3>kT5 SFa$9C2Ld0cS4Dw#!vO#$CpRqs literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/3/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/3/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..57063019503bc7c09e5adedab5929420eb2f8525 GIT binary patch literal 244 zcmeZ?GI7euPtI07#lXOj!oLlObjeG3@kGki3I=`yMom^dMxZ2*zJtvS25y!OAeE*JXBe587aE zO@WdQ0t_GYDu9gQg2bY1-ICPe65W#gg3M%Hqa+J6L(4Q%-4x?w6J3*JqZHjF(-cEp zixkTgOA}Lz#6&YApgu!Z1|}v3B>_eb1`Z|$))@>+6`V5|_*EDf7?~JYBp6s`Ffbfo z;Ge;uX{c*x7y)AO8VVf$ZotT>#lR1ACQ!gm-e?Acu>ON{GZ?sePB8KtFzSgKon!S literal 0 HcmV?d00001 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 9fe100431c01a..609cf3ce4bd75 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -28,12 +28,13 @@ import scala.collection.JavaConverters._ import scala.io.Source import scala.util.Random +import org.apache.commons.io.FileUtils import org.apache.kafka.clients.producer.{ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} +import org.apache.spark.sql.{Dataset, ForeachWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation @@ -47,6 +48,7 @@ import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils abstract class KafkaSourceTest extends StreamTest with SharedSparkSession with KafkaTest { @@ -1162,6 +1164,62 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { intercept[IllegalArgumentException] { test(minPartitions = "-1", 1, true) } } + test("default config of includeHeader doesn't break existing query from Spark 2.4") { + import testImplicits._ + + // This topic name is migrated from Spark 2.4.3 test run + val topic = "spark-test-topic-2b8619f5-d3c4-4c2d-b5d1-8d9d9458aa62" + // create same topic and messages as test run + testUtils.createTopic(topic, partitions = 5, overwrite = true) + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + (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)) } + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", topic) + .option("startingOffsets", "earliest") + .load() + + val query = kafka.dropDuplicates() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + .map(kv => kv._2.toInt + 1) + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + testStream(query)( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath), + /* + Note: The checkpoint was generated using the following input in Spark version 2.4.3 + testUtils.createTopic(topic, partitions = 5, overwrite = true) + + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + */ + makeSureGetOffsetCalled, + CheckNewAnswer(32, 33, 34, 35, 36) + ) + } } abstract class KafkaSourceSuiteBase extends KafkaSourceTest { @@ -1414,7 +1472,9 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { val now = System.currentTimeMillis() val topic = newTopic() testUtils.createTopic(newTopic(), partitions = 1) - testUtils.sendMessages(topic, Array(1).map(_.toString)) + testUtils.sendMessage( + topic, ("1", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), None + ) val kafka = spark .readStream @@ -1423,6 +1483,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { .option("kafka.metadata.max.age.ms", "1") .option("startingOffsets", s"earliest") .option("subscribe", topic) + .option("includeHeaders", "true") .load() val query = kafka @@ -1445,6 +1506,21 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { // producer. So here we just use a low bound to make sure the internal conversion works. assert(row.getAs[java.sql.Timestamp]("timestamp").getTime >= now, s"Unexpected results: $row") assert(row.getAs[Int]("timestampType") === 0, s"Unexpected results: $row") + + def checkHeader(row: Row, expected: Seq[(String, Array[Byte])]): Unit = { + // array> + val headers = row.getList[Row](row.fieldIndex("headers")).asScala + assert(headers.length === expected.length) + + (0 until expected.length).foreach { idx => + val key = headers(idx).getAs[String]("key") + val value = headers(idx).getAs[Array[Byte]]("value") + assert(key === expected(idx)._1) + assert(value === expected(idx)._2) + } + } + + checkHeader(row, Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))) query.stop() } From 95073fb62b646c3e8394941c5835a396b9d48c0f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 17 Sep 2019 11:09:55 +0900 Subject: [PATCH 12/87] [SPARK-29008][SQL] Define an individual method for each common subexpression in HashAggregateExec ### What changes were proposed in this pull request? This pr proposes to define an individual method for each common subexpression in HashAggregateExec. In the current master, the common subexpr elimination code in HashAggregateExec is expanded in a single method; https://github.com/apache/spark/blob/4664a082c2c7ac989e818958c465c72833d3ccfe/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L397 The method size can be too big for JIT compilation, so I believe splitting it is beneficial for performance. For example, in a query `SELECT SUM(a + b), AVG(a + b + c) FROM VALUES (1, 1, 1) t(a, b, c)`, the current master generates; ``` /* 098 */ private void agg_doConsume_0(InternalRow localtablescan_row_0, int agg_expr_0_0, int agg_expr_1_0, int agg_expr_2_0) throws java.io.IOException { /* 099 */ // do aggregate /* 100 */ // common sub-expressions /* 101 */ int agg_value_6 = -1; /* 102 */ /* 103 */ agg_value_6 = agg_expr_0_0 + agg_expr_1_0; /* 104 */ /* 105 */ int agg_value_5 = -1; /* 106 */ /* 107 */ agg_value_5 = agg_value_6 + agg_expr_2_0; /* 108 */ boolean agg_isNull_4 = false; /* 109 */ long agg_value_4 = -1L; /* 110 */ if (!false) { /* 111 */ agg_value_4 = (long) agg_value_5; /* 112 */ } /* 113 */ int agg_value_10 = -1; /* 114 */ /* 115 */ agg_value_10 = agg_expr_0_0 + agg_expr_1_0; /* 116 */ // evaluate aggregate functions and update aggregation buffers /* 117 */ agg_doAggregate_sum_0(agg_value_10); /* 118 */ agg_doAggregate_avg_0(agg_value_4, agg_isNull_4); /* 119 */ /* 120 */ } ``` On the other hand, this pr generates; ``` /* 121 */ private void agg_doConsume_0(InternalRow localtablescan_row_0, int agg_expr_0_0, int agg_expr_1_0, int agg_expr_2_0) throws java.io.IOException { /* 122 */ // do aggregate /* 123 */ // common sub-expressions /* 124 */ long agg_subExprValue_0 = agg_subExpr_0(agg_expr_2_0, agg_expr_0_0, agg_expr_1_0); /* 125 */ int agg_subExprValue_1 = agg_subExpr_1(agg_expr_0_0, agg_expr_1_0); /* 126 */ // evaluate aggregate functions and update aggregation buffers /* 127 */ agg_doAggregate_sum_0(agg_subExprValue_1); /* 128 */ agg_doAggregate_avg_0(agg_subExprValue_0); /* 129 */ /* 130 */ } ``` I run some micro benchmarks for this pr; ``` (base) maropu~:$system_profiler SPHardwareDataType Hardware: Hardware Overview: Processor Name: Intel Core i5 Processor Speed: 2 GHz Number of Processors: 1 Total Number of Cores: 2 L2 Cache (per Core): 256 KB L3 Cache: 4 MB Memory: 8 GB (base) maropu~:$java -version java version "1.8.0_181" Java(TM) SE Runtime Environment (build 1.8.0_181-b13) Java HotSpot(TM) 64-Bit Server VM (build 25.181-b13, mixed mode) (base) maropu~:$ /bin/spark-shell --master=local[1] --conf spark.driver.memory=8g --conf spark.sql.shurtitions=1 -v val numCols = 40 val colExprs = "id AS key" +: (0 until numCols).map { i => s"id AS _c$i" } spark.range(3000000).selectExpr(colExprs: _*).createOrReplaceTempView("t") val aggExprs = (2 until numCols).map { i => (0 until i).map(d => s"_c$d") .mkString("AVG(", " + ", ")") } // Drops the time of a first run then pick that of a second run timer { sql(s"SELECT ${aggExprs.mkString(", ")} FROM t").write.format("noop").save() } // the master maxCodeGen: 12957 Elapsed time: 36.309858661s // this pr maxCodeGen=4184 Elapsed time: 2.399490285s ``` ### Why are the changes needed? To avoid the too-long-function issue in JVMs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added tests in `WholeStageCodegenSuite` Closes #25710 from maropu/SplitSubexpr. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../expressions/codegen/CodeGenerator.scala | 78 ++++++++++++++++--- .../aggregate/HashAggregateExec.scala | 10 +-- .../execution/WholeStageCodegenSuite.scala | 25 +++++- 3 files changed, 93 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 89277706233d7..1d010ff5abfcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -112,7 +112,7 @@ private[codegen] case class NewFunctionSpec( * A context for codegen, tracking a list of objects that could be passed into generated Java * function. */ -class CodegenContext { +class CodegenContext extends Logging { import CodeGenerator._ @@ -1038,13 +1038,67 @@ class CodegenContext { // Get all the expressions that appear at least twice and set up the state for subexpression // elimination. val commonExprs = equivalentExpressions.getAllEquivalentExprs.filter(_.size > 1) - val codes = commonExprs.map { e => - val expr = e.head - // Generate the code for this expression tree. - val eval = expr.genCode(this) - val state = SubExprEliminationState(eval.isNull, eval.value) - e.foreach(localSubExprEliminationExprs.put(_, state)) - eval.code.toString + val commonExprVals = commonExprs.map(_.head.genCode(this)) + + lazy val nonSplitExprCode = { + commonExprs.zip(commonExprVals).map { case (exprs, eval) => + // Generate the code for this expression tree. + val state = SubExprEliminationState(eval.isNull, eval.value) + exprs.foreach(localSubExprEliminationExprs.put(_, state)) + eval.code.toString + } + } + + val codes = if (commonExprVals.map(_.code.length).sum > SQLConf.get.methodSplitThreshold) { + if (commonExprs.map(calculateParamLength).forall(isValidParamLength)) { + commonExprs.zipWithIndex.map { case (exprs, i) => + val expr = exprs.head + val eval = commonExprVals(i) + + val isNullLiteral = eval.isNull match { + case TrueLiteral | FalseLiteral => true + case _ => false + } + val (isNull, isNullEvalCode) = if (!isNullLiteral) { + val v = addMutableState(JAVA_BOOLEAN, "subExprIsNull") + (JavaCode.isNullGlobal(v), s"$v = ${eval.isNull};") + } else { + (eval.isNull, "") + } + + // Generate the code for this expression tree and wrap it in a function. + val fnName = freshName("subExpr") + val inputVars = getLocalInputVariableValues(this, expr).toSeq + val argList = inputVars.map(v => s"${v.javaType.getName} ${v.variableName}") + val returnType = javaType(expr.dataType) + val fn = + s""" + |private $returnType $fnName(${argList.mkString(", ")}) { + | ${eval.code} + | $isNullEvalCode + | return ${eval.value}; + |} + """.stripMargin + + val value = freshName("subExprValue") + val state = SubExprEliminationState(isNull, JavaCode.variable(value, expr.dataType)) + exprs.foreach(localSubExprEliminationExprs.put(_, state)) + val inputVariables = inputVars.map(_.variableName).mkString(", ") + s"$returnType $value = ${addNewFunction(fnName, fn)}($inputVariables);" + } + } else { + val errMsg = "Failed to split subexpression code into small functions because the " + + "parameter length of at least one split function went over the JVM limit: " + + MAX_JVM_METHOD_PARAMS_LENGTH + if (Utils.isTesting) { + throw new IllegalStateException(errMsg) + } else { + logInfo(errMsg) + nonSplitExprCode + } + } + } else { + nonSplitExprCode } SubExprCodes(codes, localSubExprEliminationExprs.toMap) } @@ -1646,7 +1700,7 @@ object CodeGenerator extends Logging { def getLocalInputVariableValues( ctx: CodegenContext, expr: Expression, - subExprs: Map[Expression, SubExprEliminationState]): Set[VariableValue] = { + subExprs: Map[Expression, SubExprEliminationState] = Map.empty): Set[VariableValue] = { val argSet = mutable.Set[VariableValue]() if (ctx.INPUT_ROW != null) { argSet += JavaCode.variable(ctx.INPUT_ROW, classOf[InternalRow]) @@ -1801,6 +1855,10 @@ object CodeGenerator extends Logging { * length less than a pre-defined constant. */ def isValidParamLength(paramLength: Int): Boolean = { - paramLength <= MAX_JVM_METHOD_PARAMS_LENGTH + // This config is only for testing + SQLConf.get.getConfString("spark.sql.CodeGenerator.validParamLength", null) match { + case null | "" => paramLength <= MAX_JVM_METHOD_PARAMS_LENGTH + case validLength => paramLength <= validLength.toInt + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 7ead180d869b2..2d187e3c9ebe5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -261,14 +261,6 @@ case class HashAggregateExec( """.stripMargin } - private def isValidParamLength(paramLength: Int): Boolean = { - // This config is only for testing - sqlContext.getConf("spark.sql.HashAggregateExec.validParamLength", null) match { - case null | "" => CodeGenerator.isValidParamLength(paramLength) - case validLength => paramLength <= validLength.toInt - } - } - // Splits aggregate code into small functions because the most of JVM implementations // can not compile too long functions. Returns None if we are not able to split the given code. // @@ -294,7 +286,7 @@ case class HashAggregateExec( val paramLength = CodeGenerator.calculateParamLengthFromExprValues(inputVarsForOneFunc) // Checks if a parameter length for the `aggExprsForOneFunc` does not go over the JVM limit - if (isValidParamLength(paramLength)) { + if (CodeGenerator.isValidParamLength(paramLength)) { Some(inputVarsForOneFunc) } else { None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 6dfb6c85dc9e0..afe9eb5c151d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -403,7 +403,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { withSQLConf( SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "true", SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1", - "spark.sql.HashAggregateExec.validParamLength" -> "0") { + "spark.sql.CodeGenerator.validParamLength" -> "0") { withTable("t") { val expectedErrMsg = "Failed to split aggregate code into small functions" Seq( @@ -419,4 +419,27 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { } } } + + test("Give up splitting subexpression code if a parameter length goes over the limit") { + withSQLConf( + SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "false", + SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1", + "spark.sql.CodeGenerator.validParamLength" -> "0") { + withTable("t") { + val expectedErrMsg = "Failed to split subexpression code into small functions" + Seq( + // Test case without keys + "SELECT AVG(a + b), SUM(a + b + c) FROM VALUES((1, 1, 1)) t(a, b, c)", + // Tet case with keys + "SELECT k, AVG(a + b), SUM(a + b + c) FROM VALUES((1, 1, 1, 1)) t(k, a, b, c) " + + "GROUP BY k").foreach { query => + val e = intercept[Exception] { + sql(query).collect + }.getCause + assert(e.isInstanceOf[IllegalStateException]) + assert(e.getMessage.contains(expectedErrMsg)) + } + } + } + } } From dffd92e9779021fa7df2ec962c9cd07e0dbfc2f3 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Sep 2019 11:06:10 +0800 Subject: [PATCH 13/87] [SPARK-29100][SQL] Fix compilation error in codegen with switch from InSet expression ### What changes were proposed in this pull request? When InSet generates Java switch-based code, if the input set is empty, we don't generate switch condition, but a simple expression that is default case of original switch. ### Why are the changes needed? SPARK-26205 adds an optimization to InSet that generates Java switch condition for certain cases. When the given set is empty, it is possibly that codegen causes compilation error: ``` [info] - SPARK-29100: InSet with empty input set *** FAILED *** (58 milliseconds) [info] Code generation of input[0, int, true] INSET () failed: [info] org.codehaus.janino.InternalCompilerException: failed to compile: org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass" in "generated.java": Compiling "apply(java.lang.Object _i)"; apply(java.lang.Object _i): Operand stack inconsistent at offset 45: Previous size 0, now 1 [info] org.codehaus.janino.InternalCompilerException: failed to compile: org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass" in "generated.java": Compiling "apply(java.lang.Object _i)"; apply(java.lang.Object _i): Operand stack inconsistent at offset 45: Previous size 0, now 1 [info] at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1308) [info] at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1386) [info] at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1383) ``` ### Does this PR introduce any user-facing change? Yes. Previously, when users have InSet against an empty set, generated code causes compilation error. This patch fixed it. ### How was this patch tested? Unit test added. Closes #25806 from viirya/SPARK-29100. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 18 +++++++++++++----- .../catalyst/expressions/PredicateSuite.scala | 6 ++++++ 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 7ce113120e4c6..4c0998412f729 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -457,17 +457,25 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with break; """) + val switchCode = if (caseBranches.size > 0) { + code""" + switch (${valueGen.value}) { + ${caseBranches.mkString("\n")} + default: + ${ev.isNull} = $hasNull; + } + """ + } else { + s"${ev.isNull} = $hasNull;" + } + ev.copy(code = code""" ${valueGen.code} ${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${valueGen.isNull}; ${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false; if (!${valueGen.isNull}) { - switch (${valueGen.value}) { - ${caseBranches.mkString("\n")} - default: - ${ev.isNull} = $hasNull; - } + $switchCode } """) } 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 7bff277c793ea..b05e49ea33015 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 @@ -560,4 +560,10 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { assert(msg.contains("argument 1 requires boolean type")) } } + + test("SPARK-29100: InSet with empty input set") { + val row = create_row(1) + val inSet = InSet(BoundReference(0, IntegerType, true), Set.empty) + checkEvaluation(inSet, false, row) + } } From 4d27a259087258492d0a66ca1ace7ef584c72a6f Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 17 Sep 2019 11:52:20 +0800 Subject: [PATCH 14/87] [SPARK-22797][ML][PYTHON] Bucketizer support multi-column ### What changes were proposed in this pull request? Bucketizer support multi-column in the python side ### Why are the changes needed? Bucketizer should support multi-column like the scala side. ### Does this PR introduce any user-facing change? yes, this PR add new Python API ### How was this patch tested? added testsuites Closes #25801 from zhengruifeng/20542_py. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- python/pyspark/ml/feature.py | 102 ++++++++++++++++++++------ python/pyspark/ml/param/__init__.py | 10 +++ python/pyspark/ml/tests/test_param.py | 9 +++ 3 files changed, 98 insertions(+), 23 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9ab4e4d68691d..5f55e48ce9f45 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -327,26 +327,34 @@ class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable) @inherit_doc -class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, - JavaMLReadable, JavaMLWritable): +class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, + HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ - Maps a column of continuous features to a column of feature buckets. - - >>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)] - >>> df = spark.createDataFrame(values, ["values"]) + Maps a column of continuous features to a column of feature buckets. Since 3.0.0, + :py:class:`Bucketizer` can map multiple columns at once by setting the :py:attr:`inputCols` + parameter. Note that when both the :py:attr:`inputCol` and :py:attr:`inputCols` parameters + are set, an Exception will be thrown. The :py:attr:`splits` parameter is only used for single + column usage, and :py:attr:`splitsArray` is for multiple columns. + + >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), + ... (float("nan"), 1.0), (float("nan"), 0.0)] + >>> df = spark.createDataFrame(values, ["values1", "values2"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], - ... inputCol="values", outputCol="buckets") + ... inputCol="values1", outputCol="buckets") >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df).collect() - >>> len(bucketed) - 6 - >>> bucketed[0].buckets - 0.0 - >>> bucketed[1].buckets - 0.0 - >>> bucketed[2].buckets - 1.0 - >>> bucketed[3].buckets - 2.0 + >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df.select("values1")) + >>> bucketed.show(truncate=False) + +-------+-------+ + |values1|buckets| + +-------+-------+ + |0.1 |0.0 | + |0.4 |0.0 | + |1.2 |1.0 | + |1.5 |2.0 | + |NaN |3.0 | + |NaN |3.0 | + +-------+-------+ + ... >>> bucketizer.setParams(outputCol="b").transform(df).head().b 0.0 >>> bucketizerPath = temp_path + "/bucketizer" @@ -357,6 +365,22 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 + >>> bucketizer2 = Bucketizer(splitsArray= + ... [[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]], + ... inputCols=["values1", "values2"], outputCols=["buckets1", "buckets2"]) + >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df) + >>> bucketed2.show(truncate=False) + +-------+-------+--------+--------+ + |values1|values2|buckets1|buckets2| + +-------+-------+--------+--------+ + |0.1 |0.0 |0.0 |0.0 | + |0.4 |1.0 |0.0 |1.0 | + |1.2 |1.3 |1.0 |1.0 | + |1.5 |NaN |2.0 |2.0 | + |NaN |1.0 |3.0 |1.0 | + |NaN |0.0 |3.0 |0.0 | + +-------+-------+--------+--------+ + ... .. versionadded:: 1.4.0 """ @@ -374,14 +398,30 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries " "containing NaN values. Values outside the splits will always be treated " "as errors. Options are 'skip' (filter out rows with invalid values), " + - "'error' (throw an error), or 'keep' (keep invalid values in a special " + - "additional bucket).", + "'error' (throw an error), or 'keep' (keep invalid values in a " + + "special additional bucket). Note that in the multiple column " + + "case, the invalid handling is applied to all columns. That said " + + "for 'error' it will throw an error if any invalids are found in " + + "any column, for 'skip' it will skip rows with any invalids in " + + "any columns, etc.", typeConverter=TypeConverters.toString) + splitsArray = Param(Params._dummy(), "splitsArray", "The array of split points for mapping " + + "continuous features into buckets for multiple columns. For each input " + + "column, with n+1 splits, there are n buckets. A bucket defined by " + + "splits x,y holds values in the range [x,y) except the last bucket, " + + "which also includes y. The splits should be of length >= 3 and " + + "strictly increasing. Values at -inf, inf must be explicitly provided " + + "to cover all Double values; otherwise, values outside the splits " + + "specified will be treated as errors.", + typeConverter=TypeConverters.toListListFloat) + @keyword_only - def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"): + def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None): """ - __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error") + __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \ + splitsArray=None, inputCols=None, outputCols=None) """ super(Bucketizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Bucketizer", self.uid) @@ -391,9 +431,11 @@ def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="er @keyword_only @since("1.4.0") - def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"): + def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None): """ - setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error") + setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \ + splitsArray=None, inputCols=None, outputCols=None) Sets params for this Bucketizer. """ kwargs = self._input_kwargs @@ -413,6 +455,20 @@ def getSplits(self): """ return self.getOrDefault(self.splits) + @since("3.0.0") + def setSplitsArray(self, value): + """ + Sets the value of :py:attr:`splitsArray`. + """ + return self._set(splitsArray=value) + + @since("3.0.0") + def getSplitsArray(self): + """ + Gets the array of split points or its default value. + """ + return self.getOrDefault(self.splitsArray) + class _CountVectorizerParams(JavaParams, HasInputCol, HasOutputCol): """ diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 043c25cf9feb4..5b6b70292f099 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -134,6 +134,16 @@ def toListFloat(value): return [float(v) for v in value] raise TypeError("Could not convert %s to list of floats" % value) + @staticmethod + def toListListFloat(value): + """ + Convert a value to list of list of floats, if possible. + """ + if TypeConverters._can_convert_to_list(value): + value = TypeConverters.toList(value) + return [TypeConverters.toListFloat(v) for v in value] + raise TypeError("Could not convert %s to list of list of floats" % value) + @staticmethod def toListInt(value): """ diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index cbeac0b1319c8..72b146054ca44 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -87,6 +87,15 @@ def test_list_float(self): self.assertTrue(all([type(v) == float for v in b.getSplits()])) self.assertRaises(TypeError, lambda: Bucketizer(splits=["a", 1.0])) + def test_list_list_float(self): + b = Bucketizer(splitsArray=[[-0.1, 0.5, 3], [-5, 1.5]]) + self.assertEqual(b.getSplitsArray(), [[-0.1, 0.5, 3.0], [-5.0, 1.5]]) + self.assertTrue(all([type(v) == list for v in b.getSplitsArray()])) + self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[0]])) + self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[1]])) + self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=["a", 1.0])) + self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=[[-5, 1.5], ["a", 1.0]])) + def test_list_string(self): for labels in [np.array(['a', u'b']), ['a', u'b'], np.array(['a', 'b'])]: idx_to_string = IndexToString(labels=labels) From c8628354b7d2e6116b2a6eb3bdb2fc957c91fd03 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 17 Sep 2019 14:04:23 +0800 Subject: [PATCH 15/87] [SPARK-28996][SQL][TESTS] Add tests regarding username of HiveClient ### What changes were proposed in this pull request? This patch proposes to add new tests to test the username of HiveClient to prevent changing the semantic unintentionally. The owner of Hive table has been changed back-and-forth, principal -> username -> principal, and looks like the change is not intentional. (Please refer [SPARK-28996](https://issues.apache.org/jira/browse/SPARK-28996) for more details.) This patch intends to prevent this. This patch also renames previous HiveClientSuite(s) to HivePartitionFilteringSuite(s) as it was commented as TODO, as well as previous tests are too narrowed to test only partition filtering. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Newly added UTs. Closes #25696 from HeartSaVioR/SPARK-28996. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../spark/sql/hive/client/HiveClient.scala | 2 + .../sql/hive/client/HiveClientImpl.scala | 2 +- .../hive/client/HiveClientUserNameSuite.scala | 63 +++++++++++++++++++ .../client/HiveClientUserNameSuites.scala | 28 +++++++++ ...cala => HivePartitionFilteringSuite.scala} | 3 +- ...ala => HivePartitionFilteringSuites.scala} | 4 +- 6 files changed, 97 insertions(+), 5 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala rename sql/hive/src/test/scala/org/apache/spark/sql/hive/client/{HiveClientSuite.scala => HivePartitionFilteringSuite.scala} (99%) rename sql/hive/src/test/scala/org/apache/spark/sql/hive/client/{HiveClientSuites.scala => HivePartitionFilteringSuites.scala} (87%) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index cb015d7301c19..51b1778ec653f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -292,4 +292,6 @@ private[hive] trait HiveClient { /** Used for testing only. Removes all metadata from this instance of Hive. */ def reset(): Unit + /** Returns the user name which is used as owner for Hive table. */ + def userName: String } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index ae9eca823d00c..96e61bd542806 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -222,7 +222,7 @@ private[hive] class HiveClientImpl( hiveConf } - private val userName = UserGroupInformation.getCurrentUser.getShortUserName + override val userName = UserGroupInformation.getCurrentUser.getShortUserName override def getConf(key: String, defaultValue: String): String = { conf.get(key, defaultValue) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala new file mode 100644 index 0000000000000..77956f4fe69da --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.client + +import java.security.PrivilegedExceptionAction + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.UserGroupInformation +import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester} + +import org.apache.spark.util.Utils + +class HiveClientUserNameSuite(version: String) extends HiveVersionSuite(version) { + + test("username of HiveClient - no UGI") { + // Assuming we're not faking System username + assert(getUserNameFromHiveClient === System.getProperty("user.name")) + } + + test("username of HiveClient - UGI") { + val ugi = UserGroupInformation.createUserForTesting( + "fakeprincipal@EXAMPLE.COM", Array.empty) + ugi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + assert(getUserNameFromHiveClient === ugi.getShortUserName) + } + }) + } + + test("username of HiveClient - Proxy user") { + val ugi = UserGroupInformation.createUserForTesting( + "fakeprincipal@EXAMPLE.COM", Array.empty) + val proxyUgi = UserGroupInformation.createProxyUserForTesting( + "proxyprincipal@EXAMPLE.COM", ugi, Array.empty) + proxyUgi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + assert(getUserNameFromHiveClient === proxyUgi.getShortUserName) + } + }) + } + + private def getUserNameFromHiveClient: String = { + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.warehouse.dir", Utils.createTempDir().toURI().toString()) + val client = buildClient(hadoopConf) + client.userName + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala new file mode 100644 index 0000000000000..e076c01c08980 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.client + +import scala.collection.immutable.IndexedSeq + +import org.scalatest.Suite + +class HiveClientUserNameSuites extends Suite with HiveClientVersions { + override def nestedSuites: IndexedSeq[Suite] = { + versions.map(new HiveClientUserNameSuite(_)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala similarity index 99% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala index bda711200acdb..5f4ee7d7f1c0f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala @@ -31,8 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, StructType} import org.apache.spark.util.Utils -// TODO: Refactor this to `HivePartitionFilteringSuite` -class HiveClientSuite(version: String) +class HivePartitionFilteringSuite(version: String) extends HiveVersionSuite(version) with BeforeAndAfterAll { private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuites.scala similarity index 87% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuites.scala index de1be2115b2d8..a43e778b13b92 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuites.scala @@ -21,9 +21,9 @@ import scala.collection.immutable.IndexedSeq import org.scalatest.Suite -class HiveClientSuites extends Suite with HiveClientVersions { +class HivePartitionFilteringSuites extends Suite with HiveClientVersions { override def nestedSuites: IndexedSeq[Suite] = { // Hive 0.12 does not provide the partition filtering API we call - versions.filterNot(_ == "0.12").map(new HiveClientSuite(_)) + versions.filterNot(_ == "0.12").map(new HivePartitionFilteringSuite(_)) } } From db996ccad91bbd7db412b1363641820784ce77bc Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Sep 2019 16:00:10 +0900 Subject: [PATCH 16/87] [SPARK-29074][SQL] Optimize `date_format` for foldable `fmt` ### What changes were proposed in this pull request? In the PR, I propose to create an instance of `TimestampFormatter` only once at the initialization, and reuse it inside of `nullSafeEval()` and `doGenCode()` in the case when the `fmt` parameter is foldable. ### Why are the changes needed? The changes improve performance of the `date_format()` function. Before: ``` format date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ format date wholestage off 7180 / 7181 1.4 718.0 1.0X format date wholestage on 7051 / 7194 1.4 705.1 1.0X ``` After: ``` 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 ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? By existing test suites `DateExpressionsSuite` and `DateFunctionsSuite`. Closes #25782 from MaxGekk/date_format-foldable. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../expressions/datetimeExpressions.scala | 32 ++++++++++++++----- .../benchmarks/DateTimeBenchmark-results.txt | 4 +-- 2 files changed, 26 insertions(+), 10 deletions(-) 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 eafdca2bfc53e..597be5dc97b43 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 @@ -592,19 +592,35 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) + @transient private lazy val formatter: Option[TimestampFormatter] = { + if (right.foldable) { + Option(right.eval()).map(format => TimestampFormatter(format.toString, zoneId)) + } else None + } + override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val df = TimestampFormatter(format.toString, zoneId) - UTF8String.fromString(df.format(timestamp.asInstanceOf[Long])) + val tf = if (formatter.isEmpty) { + TimestampFormatter(format.toString, zoneId) + } else { + formatter.get + } + UTF8String.fromString(tf.format(timestamp.asInstanceOf[Long])) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tf = TimestampFormatter.getClass.getName.stripSuffix("$") - val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val locale = ctx.addReferenceObj("locale", Locale.US) - defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $zid, $locale) + formatter.map { tf => + val timestampFormatter = ctx.addReferenceObj("timestampFormatter", tf) + defineCodeGen(ctx, ev, (timestamp, _) => { + s"""UTF8String.fromString($timestampFormatter.format($timestamp))""" + }) + }.getOrElse { + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + defineCodeGen(ctx, ev, (timestamp, format) => { + s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $zid) .format($timestamp))""" - }) + }) + } } override def prettyName: String = "date_format" diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 1a58b05a2abba..7d562544dd498 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -168,8 +168,8 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz format date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -format date wholestage off 7180 / 7181 1.4 718.0 1.0X -format date wholestage on 7051 / 7194 1.4 705.1 1.0X +format date wholestage off 4787 / 4839 2.1 478.7 1.0X +format date wholestage on 4736 / 4802 2.1 473.6 1.0X ================================================================================================ From 79b10a1aab9be6abdf749ad94c88234ace8ba34a Mon Sep 17 00:00:00 2001 From: iRakson Date: Tue, 17 Sep 2019 00:53:12 -0700 Subject: [PATCH 17/87] [SPARK-28929][CORE] Spark Logging level should be INFO instead of DEBUG in Executor Plugin API ### What changes were proposed in this pull request? Log levels in Executor.scala are changed from DEBUG to INFO. ### Why are the changes needed? Logging level DEBUG is too low here. These messages are simple acknowledgement for successful loading and initialization of plugins. So its better to keep them in INFO level. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested. Closes #25634 from iRakson/ExecutorPlugin. Authored-by: iRakson Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c337d24381286..a32226581b973 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -139,7 +139,7 @@ private[spark] class Executor( private val executorPlugins: Seq[ExecutorPlugin] = { val pluginNames = conf.get(EXECUTOR_PLUGINS) if (pluginNames.nonEmpty) { - logDebug(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") + logInfo(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") // Plugins need to load using a class loader that includes the executor's user classpath val pluginList: Seq[ExecutorPlugin] = @@ -147,12 +147,12 @@ private[spark] class Executor( val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) plugins.foreach { plugin => plugin.init() - logDebug(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + logInfo(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) } plugins } - logDebug("Finished initializing plugins") + logInfo("Finished initializing plugins") pluginList } else { Nil From 104b9b6f8c93f341bda043852aa61ea2a1d2e21b Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 17 Sep 2019 19:08:09 +0800 Subject: [PATCH 18/87] [SPARK-28483][FOLLOW-UP] Fix flaky test in BarrierTaskContextSuite ### What changes were proposed in this pull request? I fix the test "barrier task killed" which is flaky: * Split interrupt/no interrupt test into separate sparkContext. Prevent them to influence each other. * only check exception on partiton-0. partition-1 is hang on sleep which may throw other exception. ### Why are the changes needed? Make test robust. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #25799 from WeichenXu123/oss_fix_barrier_test. Authored-by: WeichenXu Signed-off-by: WeichenXu --- .../scheduler/BarrierTaskContextSuite.scala | 72 ++++++++----------- 1 file changed, 30 insertions(+), 42 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index 8d5f04ac7651a..fc8ac38479932 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -26,13 +26,18 @@ import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { - test("global sync by barrier() call") { + def initLocalClusterSparkContext(): Unit = { val conf = new SparkConf() // Init local cluster here so each barrier task runs in a separated process, thus `barrier()` // call is actually useful. .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") + .set(TEST_NO_STAGE_RETRY, true) sc = new SparkContext(conf) + } + + test("global sync by barrier() call") { + initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -48,10 +53,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("support multiple barrier() call within a single task") { - val conf = new SparkConf() - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -77,12 +79,8 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("throw exception on barrier() call timeout") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() + sc.conf.set("spark.barrier.sync.timeout", "1") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -102,12 +100,8 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("throw exception if barrier() call doesn't happen on every task") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() + sc.conf.set("spark.barrier.sync.timeout", "1") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -125,12 +119,8 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("throw exception if the number of barrier() calls are not the same on every task") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() + sc.conf.set("spark.barrier.sync.timeout", "1") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -156,10 +146,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { assert(error.contains("within 1 second(s)")) } - - def testBarrierTaskKilled(sc: SparkContext, interruptOnCancel: Boolean): Unit = { - sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, interruptOnCancel.toString) - + def testBarrierTaskKilled(interruptOnKill: Boolean): Unit = { withTempDir { dir => val killedFlagFile = "barrier.task.killed" val rdd = sc.makeRDD(Seq(0, 1), 2) @@ -181,12 +168,15 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { val listener = new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - new Thread { - override def run: Unit = { - Thread.sleep(1000) - sc.killTaskAttempt(taskStart.taskInfo.taskId, interruptThread = false) - } - }.start() + val partitionId = taskStart.taskInfo.index + if (partitionId == 0) { + new Thread { + override def run: Unit = { + Thread.sleep(1000) + sc.killTaskAttempt(taskStart.taskInfo.taskId, interruptThread = interruptOnKill) + } + }.start() + } } } sc.addSparkListener(listener) @@ -201,15 +191,13 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } } - test("barrier task killed") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + test("barrier task killed, no interrupt") { + initLocalClusterSparkContext() + testBarrierTaskKilled(interruptOnKill = false) + } - testBarrierTaskKilled(sc, true) - testBarrierTaskKilled(sc, false) + test("barrier task killed, interrupt") { + initLocalClusterSparkContext() + testBarrierTaskKilled(interruptOnKill = true) } } From 34915b22ab174a45c563ccdcd5035299f3ccc56c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 Sep 2019 20:21:25 +0900 Subject: [PATCH 19/87] [SPARK-29104][CORE][TESTS] Fix PipedRDDSuite to use `eventually` to check thread termination ### What changes were proposed in this pull request? `PipedRDD` will invoke `stdinWriterThread.interrupt()` at task completion, and `obj.wait` will get `InterruptedException`. However, there exists a possibility which the thread termination gets delayed because the thread starts from `obj.wait()` with that exception. To prevent test flakiness, we need to use `eventually`. Also, This PR fixes the typo in code comment and variable name. ### Why are the changes needed? ``` - stdin writer thread should be exited when task is finished *** FAILED *** Some(Thread[stdin writer for List(cat),5,]) was not empty (PipedRDDSuite.scala:107) ``` - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/6867/testReport/junit/org.apache.spark.rdd/PipedRDDSuite/stdin_writer_thread_should_be_exited_when_task_is_finished/ ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual. We can reproduce the same failure like Jenkins if we catch `InterruptedException` and sleep longer than the `eventually` timeout inside the test code. The following is the example to reproduce it. ```scala val nums = sc.makeRDD(Array(1, 2, 3, 4), 1).map { x => try { obj.synchronized { obj.wait() // make the thread waits here. } } catch { case ie: InterruptedException => Thread.sleep(15000) throw ie } x } ``` Closes #25808 from dongjoon-hyun/SPARK-29104. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .../org/apache/spark/rdd/PipedRDDSuite.scala | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 69739a2e58481..7d419579a36d0 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -21,16 +21,18 @@ import java.io.File import scala.collection.JavaConverters._ import scala.collection.Map +import scala.concurrent.duration._ import scala.io.Codec import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat} +import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.util.Utils -class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { +class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { val envCommand = if (Utils.isWindows) { "cmd.exe /C set" } else { @@ -100,11 +102,16 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { assert(result.collect().length === 0) - // collect stderr writer threads - val stderrWriterThread = Thread.getAllStackTraces.keySet().asScala - .find { _.getName.startsWith(PipedRDD.STDIN_WRITER_THREAD_PREFIX) } - - assert(stderrWriterThread.isEmpty) + // SPARK-29104 PipedRDD will invoke `stdinWriterThread.interrupt()` at task completion, + // and `obj.wait` will get InterruptedException. However, there exists a possibility + // which the thread termination gets delayed because the thread starts from `obj.wait()` + // with that exception. To prevent test flakiness, we need to use `eventually`. + eventually(timeout(10.seconds), interval(1.second)) { + // collect stdin writer threads + val stdinWriterThread = Thread.getAllStackTraces.keySet().asScala + .find { _.getName.startsWith(PipedRDD.STDIN_WRITER_THREAD_PREFIX) } + assert(stdinWriterThread.isEmpty) + } } test("advanced pipe") { From 3fc52b5557b4608d8f0ce26d11c1ca3e24c157a2 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Wed, 18 Sep 2019 01:14:14 +0800 Subject: [PATCH 20/87] [SPARK-28950][SQL] Refine the code of DELETE ### What changes were proposed in this pull request? This pr refines the code of DELETE, including, 1, make `whereClause` to be optional, in which case DELETE will delete all of the data of a table; 2, add more test cases; 3, some other refines. This is a following-up of SPARK-28351. ### Why are the changes needed? An optional where clause in DELETE respects the SQL standard. ### Does this PR introduce any user-facing change? Yes. But since this is a non-released feature, this change does not have any end-user affects. ### How was this patch tested? New case is added. Closes #25652 from xianyinxin/SPARK-28950. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 14 +++++++-- .../plans/logical/basicLogicalOperators.scala | 2 +- .../logical/sql/DeleteFromStatement.scala | 2 +- .../sql/catalyst/parser/DDLParserSuite.scala | 29 +++++++++++++++++-- .../datasources/v2/DataSourceV2Strategy.scala | 13 +++++---- .../sql/connector/DataSourceV2SQLSuite.scala | 27 +++++++++++++++-- 7 files changed, 73 insertions(+), 16 deletions(-) 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 70c0d0e505f6c..20be8e539cf58 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 @@ -216,7 +216,7 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration - | DELETE FROM multipartIdentifier tableAlias whereClause #deleteFromTable + | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; 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 8556ff4775530..9335be5b239b6 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 @@ -343,12 +343,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) val tableAlias = if (ctx.tableAlias() != null) { val ident = ctx.tableAlias().strictIdentifier() - if (ident != null) { Some(ident.getText) } else { None } + // We do not allow columns aliases after table alias. + if (ctx.tableAlias().identifierList() != null) { + throw new ParseException("Columns aliases is not allowed in DELETE.", + ctx.tableAlias().identifierList()) + } + if (ident != null) Some(ident.getText) else None + } else { + None + } + val predicate = if (ctx.whereClause() != null) { + Some(expression(ctx.whereClause().booleanExpression())) } else { None } - DeleteFromStatement(tableId, tableAlias, expression(ctx.whereClause().booleanExpression())) + DeleteFromStatement(tableId, tableAlias, predicate) } /** 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 017f31760b584..2b6378e7c7268 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 @@ -579,7 +579,7 @@ case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Comm case class DeleteFromTable( child: LogicalPlan, - condition: Expression) extends Command { + condition: Option[Expression]) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala index 21e24127eee31..035bc09d31871 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala @@ -23,5 +23,5 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan case class DeleteFromStatement( tableName: Seq[String], tableAlias: Option[String], - condition: Expression) + condition: Option[Expression]) extends ParsedStatement 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 2cb01d4e8aa5c..be781081b0112 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 @@ -20,10 +20,11 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar} +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, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +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.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 @@ -764,6 +765,30 @@ class DDLParserSuite extends AnalysisTest { assert(exc.getMessage.contains("INSERT INTO ... IF NOT EXISTS")) } + test("delete from table: delete all") { + parseCompare("DELETE FROM testcat.ns1.ns2.tbl", + DeleteFromStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + None, + None)) + } + + test("delete from table: with alias and where clause") { + parseCompare("DELETE FROM testcat.ns1.ns2.tbl AS t WHERE t.a = 2", + DeleteFromStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + Some("t"), + Some(EqualTo(UnresolvedAttribute("t.a"), Literal(2))))) + } + + test("delete from table: columns aliases is not allowed") { + val exc = intercept[ParseException] { + parsePlan("DELETE FROM testcat.ns1.ns2.tbl AS t(a,b,c,d) WHERE d = 2") + } + + assert(exc.getMessage.contains("Columns aliases is not allowed in DELETE.")) + } + test("show tables") { comparePlans( parsePlan("SHOW TABLES"), 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 22100c7d3d593..6285159dad9b0 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 @@ -251,16 +251,17 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { OverwritePartitionsDynamicExec(r.table.asWritable, r.options, planLater(query)) :: Nil case DeleteFromTable(r: DataSourceV2Relation, condition) => - if (SubqueryExpression.hasSubquery(condition)) { + if (condition.exists(SubqueryExpression.hasSubquery)) { throw new AnalysisException( s"Delete by condition with subquery is not supported: $condition") } // fail if any filter cannot be converted. correctness depends on removing all matching data. - val filters = splitConjunctivePredicates(condition).map { - f => DataSourceStrategy.translateFilter(f).getOrElse( - throw new AnalysisException(s"Exec delete failed:" + - s" cannot translate expression to source filter: $f")) - }.toArray + val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, r.output) + .flatMap(splitConjunctivePredicates(_).map { + f => DataSourceStrategy.translateFilter(f).getOrElse( + throw new AnalysisException(s"Exec update failed:" + + s" cannot translate expression to source filter: $f")) + }).toArray DeleteFromTableExec(r.table.asDeletable, filters) :: Nil case WriteToContinuousDataSource(writer, query) => 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 ef484ce6ac362..9eb8e5b3b73fc 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 @@ -992,7 +992,17 @@ class DataSourceV2SQLSuite } } - test("DeleteFrom: basic") { + test("DeleteFrom: basic - delete all") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + sql(s"DELETE FROM $t") + checkAnswer(spark.table(t), Seq()) + } + } + + test("DeleteFrom: basic - delete with where clause") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") @@ -1003,12 +1013,23 @@ class DataSourceV2SQLSuite } } - test("DeleteFrom: alias") { + test("DeleteFrom: delete from aliased target table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + sql(s"DELETE FROM $t AS tbl WHERE tbl.id = 2") + checkAnswer(spark.table(t), Seq( + Row(3, "c", 3))) + } + } + + test("DeleteFrom: normalize attribute names") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") - sql(s"DELETE FROM $t tbl WHERE tbl.id = 2") + sql(s"DELETE FROM $t AS tbl WHERE tbl.ID = 2") checkAnswer(spark.table(t), Seq( Row(3, "c", 3))) } From c6ca66113f8d988a61683ee0e98038b74b7db24f Mon Sep 17 00:00:00 2001 From: sharangk Date: Tue, 17 Sep 2019 14:36:56 -0700 Subject: [PATCH 21/87] [SPARK-28814][SQL][DOC] Document SET/RESET in SQL Reference ### What changes were proposed in this pull request? Document SET/REST statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. #### Before: There was no documentation for this. #### After: **SET** ![image](https://user-images.githubusercontent.com/29914590/65037551-94a3c680-d96b-11e9-9d59-9f7af5185e06.png) ![image](https://user-images.githubusercontent.com/29914590/64858792-fb607180-d645-11e9-8a53-8cf87a166fc1.png) **RESET** ![image](https://user-images.githubusercontent.com/29914590/64859019-b12bc000-d646-11e9-8cb4-73dc21830067.png) ### How was this patch tested? Manual Review and Tested using jykyll build --serve Closes #25606 from sharangk/resetDoc. Authored-by: sharangk Signed-off-by: Xiao Li --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 18 +++++++- docs/sql-ref-syntax-aux-conf-mgmt-set.md | 49 +++++++++++++++++++++- 2 files changed, 65 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index ad2d7f9a83316..8ee61514ee4ef 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -19,4 +19,20 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty. + +### Syntax +{% highlight sql %} +RESET +{% endhighlight %} + + +### Examples +{% highlight sql %} +-- Reset all the properties specific to the current session to their default values. +RESET; +{% endhighlight %} + +### Related Statements +- [SET](sql-ref-syntax-aux-conf-mgmt-set.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set.md b/docs/sql-ref-syntax-aux-conf-mgmt-set.md index c38d68dbb4f1d..f05dde3f567ee 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-set.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-set.md @@ -19,4 +19,51 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The SET command sets a property, returns the value of an existing property or returns all SQLConf properties with value and meaning. + +### Syntax +{% highlight sql %} +SET +SET [ -v ] +SET property_key[ = property_value ] +{% endhighlight %} + +### Parameters +
    +
    -v
    +
    Outputs the key, value and meaning of existing SQLConf properties.
    +
    + +
    +
    property_key
    +
    Returns the value of specified property key.
    +
    + +
    +
    property_key=property_value
    +
    Sets the value for a given property key. If an old value exists for a given property key, then it gets overridden by the new value.
    +
    + +### Examples +{% highlight sql %} +-- Set a property. +SET spark.sql.variable.substitute=false; + +-- List all SQLConf properties with value and meaning. +SET -v; + +-- List all SQLConf properties with value for current session. +SET; + +-- List the value of specified property key. +SET spark.sql.variable.substitute; + +--------------------------------+--------+ + | key | value | + +--------------------------------+--------+ + | spark.sql.variable.substitute | false | + +--------------------------------+--------+ +{% endhighlight %} + +### Related Statements +- [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) From dd32476a8250e82df554683e195c355459d10a5d Mon Sep 17 00:00:00 2001 From: sharangk Date: Tue, 17 Sep 2019 14:40:08 -0700 Subject: [PATCH 22/87] [SPARK-28792][SQL][DOC] Document CREATE DATABASE statement in SQL Reference ### What changes were proposed in this pull request? Document CREATE DATABASE statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. ### Before: There was no documentation for this. ### After: ![image](https://user-images.githubusercontent.com/29914590/65037831-290e2900-d96c-11e9-8563-92e5379c3ad1.png) ![image](https://user-images.githubusercontent.com/29914590/64858915-55f9cd80-d646-11e9-91a9-16c52b1daa56.png) ### How was this patch tested? Manual Review and Tested using jykyll build --serve Closes #25595 from sharangk/createDbDoc. Lead-authored-by: sharangk Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/sql-ref-syntax-ddl-create-database.md | 59 +++++++++++++++++++++- 1 file changed, 58 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-create-database.md b/docs/sql-ref-syntax-ddl-create-database.md index bbcd34a6d6853..ed0bbf629b027 100644 --- a/docs/sql-ref-syntax-ddl-create-database.md +++ b/docs/sql-ref-syntax-ddl-create-database.md @@ -19,4 +19,61 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +Creates a database with the specified name. If database with the same name already exists, an exception will be thrown. + +### Syntax +{% highlight sql %} +CREATE {DATABASE | SCHEMA} [ IF NOT EXISTS ] database_name + [ COMMENT database_comment ] + [ LOCATION database_directory ] + [ WITH DBPROPERTIES (property_name=property_value [ , ...]) ] +{% endhighlight %} + +### Parameters +
    +
    database_name
    +
    Specifies the name of the database to be created.
    + +
    IF NOT EXISTS
    +
    Creates a database with the given name if it doesn't exists. If a database with the same name already exists, nothing will happen.
    + +
    database_directory
    +
    Path of the file system in which the specified database is to be created. If the specified path does not exist in the underlying file system, this command creates a directory with the path. If the location is not specified, the database will be created in the default warehouse directory, whose path is configured by the static configuration spark.sql.warehouse.dir.
    + +
    database_comment
    +
    Specifies the description for the database.
    + +
    WITH DBPROPERTIES (property_name=property_value [ , ...])
    +
    Specifies the properties for the database in key-value pairs.
    +
    + +### Examples +{% highlight sql %} +-- Create database `customer_db`. This throws exception if database with name customer_db +-- already exists. +CREATE DATABASE customer_db; + +-- Create database `customer_db` only if database with same name doesn't exist. +CREATE DATABASE IF NOT EXISTS customer_db; + +-- Create database `customer_db` only if database with same name doesn't exist with +-- `Comments`,`Specific Location` and `Database properties`. +CREATE DATABASE IF NOT EXISTS customer_db COMMENT 'This is customer database' LOCATION '/user' + WITH DBPROPERTIES (ID=001, Name='John'); + +-- Verify that properties are set. +DESCRIBE DATABASE EXTENDED customer_db; + +----------------------------+-----------------------------+ + | database_description_item | database_description_value | + +----------------------------+-----------------------------+ + | Database Name | customer_db | + | Description | This is customer database | + | Location | hdfs://hacluster/user | + | Properties | ((ID,001), (Name,John)) | + +----------------------------+-----------------------------+ +{% endhighlight %} + +### Related Statements +- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) From 02db706090c11c3ffc056be3ca1baf3affe5788a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Sep 2019 15:09:16 -0700 Subject: [PATCH 23/87] [SPARK-29115][SQL][TEST] Add benchmarks for make_date() and make_timestamp() ### What changes were proposed in this pull request? Added new benchmarks for `make_date()` and `make_timestamp()` to detect performance issues, and figure out functions speed on foldable arguments. - `make_date()` is benchmarked on fully foldable arguments. - `make_timestamp()` is benchmarked on corner case `60.0`, foldable time fields and foldable date. ### Why are the changes needed? To find out inputs where `make_date()` and `make_timestamp()` have performance problems. This should be useful in the future optimizations of the functions and users apps. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the benchmark and manually checking generated dates/timestamps. Closes #25813 from MaxGekk/make_datetime-benchmark. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../MakeDateTimeBenchmark-results.txt | 22 ++++ .../benchmark/MakeDateTimeBenchmark.scala | 120 ++++++++++++++++++ 2 files changed, 142 insertions(+) create mode 100644 sql/core/benchmarks/MakeDateTimeBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt new file mode 100644 index 0000000000000..a3c89d643e912 --- /dev/null +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -0,0 +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 +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 + +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 +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 + 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 new file mode 100644 index 0000000000000..7f7908544693f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for the make_date() and make_timestamp() functions. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "sql/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/MakeDateTimeBenchmark-results.txt". + * }}} + */ +object MakeDateTimeBenchmark extends SqlBasedBenchmark { + + private def doBenchmark(cardinality: Long, exprs: String*): Unit = { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + spark + .range(0, cardinality, 1, 1) + .selectExpr(exprs: _*) + .write + .format("noop") + .save() + } + } + + private def run(benchmark: Benchmark, cardinality: Long, name: String, exprs: String*): Unit = { + benchmark.addCase(name, numIters = 3) { _ => doBenchmark(cardinality, exprs: _*) } + } + + private val ymdExprs = Seq("(2000 + (id % 30))", "((id % 12) + 1)", "((id % 27) + 1)") + + private def benchmarkMakeDate(cardinality: Long): Unit = { + val benchmark = new Benchmark("make_date()", cardinality, output = output) + val args = ymdExprs + + run(benchmark, cardinality, "prepare make_date()", args: _*) + val foldableExpr = "make_date(2019, 9, 16)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + run( + benchmark, + cardinality, + "make_date(*, *, *)", + "make_date" + args.mkString("(", ",", ")")) + + benchmark.run() + } + + private def benchmarkMakeTimestamp(cardinality: Long): Unit = { + val benchmark = new Benchmark("make_timestamp()", cardinality, output = output) + val hmExprs = Seq("id % 24", "id % 60") + val hmsExprs = hmExprs ++ Seq("cast((id % 60000000) / 1000000.0 as decimal(8, 6))") + val args = ymdExprs ++ hmsExprs + + run( + benchmark, + cardinality, + "prepare make_timestamp()", + args: _*) + var foldableExpr = "make_timestamp(2019, 1, 2, 3, 4, 50.123456)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + foldableExpr = "make_timestamp(2019, 1, 2, 3, 4, 60.000000)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + foldableExpr = "make_timestamp(2019, 12, 31, 23, 59, 60.00)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, 3, 4, 50.123456)", + s"make_timestamp(${ymdExprs.mkString(",")}, 3, 4, 50.123456)") + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, *, *, 0)", + s"make_timestamp(" + (ymdExprs ++ hmExprs).mkString(", ") + ", 0)") + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, *, *, 60.0)", + s"make_timestamp(" + (ymdExprs ++ hmExprs).mkString(", ") + ", 60.0)") + run( + benchmark, + cardinality, + "make_timestamp(2019, 1, 2, *, *, *)", + s"make_timestamp(2019, 1, 2, ${hmsExprs.mkString(",")})") + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, *, *, *)", + s"make_timestamp" + args.mkString("(", ", ", ")")) + + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + benchmarkMakeDate(100000000L) + benchmarkMakeTimestamp(1000000L) + } +} From 71e7516132147b200db65a202b8cf74108958430 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 17 Sep 2019 15:30:18 -0700 Subject: [PATCH 24/87] [SPARK-29027][TESTS] KafkaDelegationTokenSuite fix when loopback canonical host name differs from localhost ### What changes were proposed in this pull request? `KafkaDelegationTokenSuite` fails on different platforms with the following problem: ``` 19/09/11 11:07:42.690 pool-1-thread-1-SendThread(localhost:44965) DEBUG ZooKeeperSaslClient: creating sasl client: Client=zkclient/localhostEXAMPLE.COM;service=zookeeper;serviceHostname=localhost.localdomain ... NIOServerCxn.Factory:localhost/127.0.0.1:0: Zookeeper Server failed to create a SaslServer to interact with a client during session initiation: javax.security.sasl.SaslException: Failure to initialize security context [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails)] at com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) at com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) at org.apache.zookeeper.util.SecurityUtils$2.run(SecurityUtils.java:233) at org.apache.zookeeper.util.SecurityUtils$2.run(SecurityUtils.java:229) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.zookeeper.util.SecurityUtils.createSaslServer(SecurityUtils.java:228) at org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:44) at org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:38) at org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) at org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:186) at org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:227) at java.lang.Thread.run(Thread.java:748) Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails) at sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) at sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) at sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) at com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) ... 13 more NIOServerCxn.Factory:localhost/127.0.0.1:0: Client attempting to establish new session at /127.0.0.1:33742 SyncThread:0: Creating new log file: log.1 SyncThread:0: Established session 0x100003736ae0000 with negotiated timeout 10000 for client /127.0.0.1:33742 pool-1-thread-1-SendThread(localhost:35625): Session establishment complete on server localhost/127.0.0.1:35625, sessionid = 0x100003736ae0000, negotiated timeout = 10000 pool-1-thread-1-SendThread(localhost:35625): ClientCnxn:sendSaslPacket:length=0 pool-1-thread-1-SendThread(localhost:35625): saslClient.evaluateChallenge(len=0) pool-1-thread-1-EventThread: zookeeper state changed (SyncConnected) NioProcessor-1: No server entry found for kerberos principal name zookeeper/localhost.localdomainEXAMPLE.COM NioProcessor-1: No server entry found for kerberos principal name zookeeper/localhost.localdomainEXAMPLE.COM NioProcessor-1: Server not found in Kerberos database (7) NioProcessor-1: Server not found in Kerberos database (7) ``` The problem reproducible if the `localhost` and `localhost.localdomain` order exhanged: ``` [systestgsomogyi-build spark]$ cat /etc/hosts 127.0.0.1 localhost.localdomain localhost localhost4 localhost4.localdomain4 ::1 localhost.localdomain localhost localhost6 localhost6.localdomain6 ``` The main problem is that `ZkClient` connects to the canonical loopback address (which is not necessarily `localhost`). ### Why are the changes needed? `KafkaDelegationTokenSuite` failed in some environments. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit tests on different platforms. Closes #25803 from gaborgsomogyi/SPARK-29027. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../spark/sql/kafka010/KafkaTestUtils.scala | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) 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 f7114129a3cdc..ee3eb43967069 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.kafka010 import java.io.{File, IOException} import java.lang.{Integer => JInt} -import java.net.InetSocketAddress +import java.net.{InetAddress, InetSocketAddress} import java.nio.charset.StandardCharsets import java.util.{Collections, Map => JMap, Properties, UUID} import java.util.concurrent.TimeUnit @@ -68,10 +68,13 @@ class KafkaTestUtils( private val JAVA_AUTH_CONFIG = "java.security.auth.login.config" + private val localCanonicalHostName = InetAddress.getLoopbackAddress().getCanonicalHostName() + logInfo(s"Local host name is $localCanonicalHostName") + private var kdc: MiniKdc = _ // Zookeeper related configurations - private val zkHost = "localhost" + private val zkHost = localCanonicalHostName private var zkPort: Int = 0 private val zkConnectionTimeout = 60000 private val zkSessionTimeout = 10000 @@ -80,12 +83,12 @@ class KafkaTestUtils( private var zkUtils: ZkUtils = _ // Kafka broker related configurations - private val brokerHost = "localhost" + private val brokerHost = localCanonicalHostName private var brokerPort = 0 private var brokerConf: KafkaConfig = _ private val brokerServiceName = "kafka" - private val clientUser = "client/localhost" + private val clientUser = s"client/$localCanonicalHostName" private var clientKeytabFile: File = _ // Kafka broker server @@ -139,17 +142,17 @@ class KafkaTestUtils( assert(kdcReady, "KDC should be set up beforehand") val baseDir = Utils.createTempDir() - val zkServerUser = "zookeeper/localhost" + val zkServerUser = s"zookeeper/$localCanonicalHostName" val zkServerKeytabFile = new File(baseDir, "zookeeper.keytab") kdc.createPrincipal(zkServerKeytabFile, zkServerUser) logDebug(s"Created keytab file: ${zkServerKeytabFile.getAbsolutePath()}") - val zkClientUser = "zkclient/localhost" + val zkClientUser = s"zkclient/$localCanonicalHostName" val zkClientKeytabFile = new File(baseDir, "zkclient.keytab") kdc.createPrincipal(zkClientKeytabFile, zkClientUser) logDebug(s"Created keytab file: ${zkClientKeytabFile.getAbsolutePath()}") - val kafkaServerUser = "kafka/localhost" + val kafkaServerUser = s"kafka/$localCanonicalHostName" val kafkaServerKeytabFile = new File(baseDir, "kafka.keytab") kdc.createPrincipal(kafkaServerKeytabFile, kafkaServerUser) logDebug(s"Created keytab file: ${kafkaServerKeytabFile.getAbsolutePath()}") From 197732e1f4b514a323bccb3f561d1502faeafd0a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 Sep 2019 16:53:21 -0700 Subject: [PATCH 25/87] [SPARK-29125][INFRA] Add Hadoop 2.7 combination to GitHub Action ### What changes were proposed in this pull request? Until now, we are testing JDK8/11 with Hadoop-3.2. This PR aims to extend the test coverage for JDK8/Hadoop-2.7. ### Why are the changes needed? This will prevent Hadoop 2.7 compile/package issues at PR stage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? GitHub Action on this PR shows all three combinations now. And, this is irrelevant to Jenkins test. Closes #25824 from dongjoon-hyun/SPARK-29125. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index a4233d113a561..dfed834c1d08e 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -15,7 +15,11 @@ jobs: strategy: matrix: java: [ '1.8', '11' ] - name: Build Spark with JDK ${{ matrix.java }} + hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ] + exclude: + - java: '11' + hadoop: 'hadoop-2.7' + name: Build Spark with JDK ${{ matrix.java }} and ${{ matrix.hadoop }} steps: - uses: actions/checkout@master @@ -27,4 +31,4 @@ jobs: run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-3.2 -Phadoop-cloud -Djava.version=${{ matrix.java }} package + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} package From 05988b256e8d6d07517a9b0d037649fa8c3fc65d Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 17 Sep 2019 17:13:50 -0700 Subject: [PATCH 26/87] [SPARK-27463][PYTHON] Support Dataframe Cogroup via Pandas UDFs ### What changes were proposed in this pull request? Adds a new cogroup Pandas UDF. This allows two grouped dataframes to be cogrouped together and apply a (pandas.DataFrame, pandas.DataFrame) -> pandas.DataFrame UDF to each cogroup. **Example usage** ``` from pyspark.sql.functions import pandas_udf, PandasUDFType df1 = spark.createDataFrame( [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], ("time", "id", "v1")) df2 = spark.createDataFrame( [(20000101, 1, "x"), (20000101, 2, "y")], ("time", "id", "v2")) pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) def asof_join(l, r): return pd.merge_asof(l, r, on="time", by="id") df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() ``` +--------+---+---+---+ | time| id| v1| v2| +--------+---+---+---+ |20000101| 1|1.0| x| |20000102| 1|3.0| x| |20000101| 2|2.0| y| |20000102| 2|4.0| y| +--------+---+---+---+ ### How was this patch tested? Added unit test test_pandas_udf_cogrouped_map Closes #24981 from d80tb7/SPARK-27463-poc-arrow-stream. Authored-by: Chris Martin Signed-off-by: Bryan Cutler --- .../spark/api/python/PythonRunner.scala | 2 + python/pyspark/rdd.py | 1 + python/pyspark/serializers.py | 26 ++ python/pyspark/sql/cogroup.py | 98 ++++++ python/pyspark/sql/functions.py | 5 +- python/pyspark/sql/group.py | 12 +- .../tests/test_pandas_udf_cogrouped_map.py | 280 ++++++++++++++++++ python/pyspark/sql/udf.py | 19 ++ python/pyspark/worker.py | 98 +++++- .../sql/catalyst/analysis/Analyzer.scala | 6 + .../logical/pythonLogicalOperators.scala | 19 +- .../spark/sql/RelationalGroupedDataset.scala | 46 ++- .../spark/sql/execution/SparkStrategies.scala | 3 + .../execution/python/ArrowPythonRunner.scala | 76 +---- .../python/BaseArrowPythonRunner.scala | 112 +++++++ .../python/BasePandasGroupExec.scala | 137 +++++++++ .../python/CogroupedArrowPythonRunner.scala | 113 +++++++ .../python/FlatMapCoGroupsInPandasExec.scala | 97 ++++++ .../python/FlatMapGroupsInPandasExec.scala | 98 +----- 19 files changed, 1070 insertions(+), 178 deletions(-) create mode 100644 python/pyspark/sql/cogroup.py create mode 100644 python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index d2a10df7acbd3..277804ec41d98 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -48,6 +48,7 @@ private[spark] object PythonEvalType { val SQL_WINDOW_AGG_PANDAS_UDF = 203 val SQL_SCALAR_PANDAS_ITER_UDF = 204 val SQL_MAP_PANDAS_ITER_UDF = 205 + val SQL_COGROUPED_MAP_PANDAS_UDF = 206 def toString(pythonEvalType: Int): String = pythonEvalType match { case NON_UDF => "NON_UDF" @@ -58,6 +59,7 @@ private[spark] object PythonEvalType { case SQL_WINDOW_AGG_PANDAS_UDF => "SQL_WINDOW_AGG_PANDAS_UDF" case SQL_SCALAR_PANDAS_ITER_UDF => "SQL_SCALAR_PANDAS_ITER_UDF" case SQL_MAP_PANDAS_ITER_UDF => "SQL_MAP_PANDAS_ITER_UDF" + case SQL_COGROUPED_MAP_PANDAS_UDF => "SQL_COGROUPED_MAP_PANDAS_UDF" } } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 16c226f02e633..be0244b7d13e2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -75,6 +75,7 @@ class PythonEvalType(object): SQL_WINDOW_AGG_PANDAS_UDF = 203 SQL_SCALAR_PANDAS_ITER_UDF = 204 SQL_MAP_PANDAS_ITER_UDF = 205 + SQL_COGROUPED_MAP_PANDAS_UDF = 206 def portable_hash(x): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 00f6081a3b14f..bceb92cb274ae 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -401,6 +401,32 @@ def __repr__(self): return "ArrowStreamPandasUDFSerializer" +class CogroupUDFSerializer(ArrowStreamPandasUDFSerializer): + + def load_stream(self, stream): + """ + Deserialize Cogrouped ArrowRecordBatches to a tuple of Arrow tables and yield as two + lists of pandas.Series. + """ + import pyarrow as pa + dataframes_in_group = None + + while dataframes_in_group is None or dataframes_in_group > 0: + dataframes_in_group = read_int(stream) + + if dataframes_in_group == 2: + batch1 = [batch for batch in ArrowStreamSerializer.load_stream(self, stream)] + batch2 = [batch for batch in ArrowStreamSerializer.load_stream(self, stream)] + yield ( + [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch1).itercolumns()], + [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch2).itercolumns()] + ) + + elif dataframes_in_group != 0: + raise ValueError( + 'Invalid number of pandas.DataFrames in group {0}'.format(dataframes_in_group)) + + class BatchedSerializer(Serializer): """ diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py new file mode 100644 index 0000000000000..9b725e4bafe79 --- /dev/null +++ b/python/pyspark/sql/cogroup.py @@ -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. +# + +from pyspark import since +from pyspark.rdd import PythonEvalType +from pyspark.sql.column import Column +from pyspark.sql.dataframe import DataFrame + + +class CoGroupedData(object): + """ + A logical grouping of two :class:`GroupedData`, + created by :func:`GroupedData.cogroup`. + + .. note:: Experimental + + .. versionadded:: 3.0 + """ + + def __init__(self, gd1, gd2): + self._gd1 = gd1 + self._gd2 = gd2 + self.sql_ctx = gd1.sql_ctx + + @since(3.0) + def apply(self, udf): + """ + Applies a function to each cogroup using a pandas udf and returns the result + as a `DataFrame`. + + The user-defined function should take two `pandas.DataFrame` and return another + `pandas.DataFrame`. For each side of the cogroup, all columns are passed together + as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` + are combined as a :class:`DataFrame`. + + The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the + returnType of the pandas udf. + + .. note:: This function requires a full shuffle. All the data of a cogroup will be loaded + into memory, so the user should be aware of the potential OOM risk if data is skewed + and certain groups are too large to fit in memory. + + .. note:: Experimental + + :param udf: a cogrouped map user-defined function returned by + :func:`pyspark.sql.functions.pandas_udf`. + + >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> df1 = spark.createDataFrame( + ... [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], + ... ("time", "id", "v1")) + >>> df2 = spark.createDataFrame( + ... [(20000101, 1, "x"), (20000101, 2, "y")], + ... ("time", "id", "v2")) + >>> @pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) + ... def asof_join(l, r): + ... return pd.merge_asof(l, r, on="time", by="id") + >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() + +--------+---+---+---+ + | time| id| v1| v2| + +--------+---+---+---+ + |20000101| 1|1.0| x| + |20000102| 1|3.0| x| + |20000101| 2|2.0| y| + |20000102| 2|4.0| y| + +--------+---+---+---+ + + .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` + + """ + # Columns are special because hasattr always return True + if isinstance(udf, Column) or not hasattr(udf, 'func') \ + or udf.evalType != PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " + "COGROUPED_MAP.") + all_cols = self._extract_cols(self._gd1) + self._extract_cols(self._gd2) + udf_column = udf(*all_cols) + jdf = self._gd1._jgd.flatMapCoGroupsInPandas(self._gd2._jgd, udf_column._jc.expr()) + return DataFrame(jdf, self.sql_ctx) + + @staticmethod + def _extract_cols(gd): + df = gd._df + return [df[col] for col in df.columns] diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index c7ff2882ed95a..d96c264aa7398 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2814,6 +2814,8 @@ class PandasUDFType(object): GROUPED_MAP = PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF + COGROUPED_MAP = PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF + GROUPED_AGG = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF MAP_ITER = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF @@ -3320,7 +3322,8 @@ def pandas_udf(f=None, returnType=None, functionType=None): PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, - PythonEvalType.SQL_MAP_PANDAS_ITER_UDF]: + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]: raise ValueError("Invalid functionType: " "functionType must be one the values from PandasUDFType") diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index ec90ba905ef66..fcad64142485e 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -22,6 +22,7 @@ from pyspark.sql.column import Column, _to_seq from pyspark.sql.dataframe import DataFrame from pyspark.sql.types import * +from pyspark.sql.cogroup import CoGroupedData __all__ = ["GroupedData"] @@ -218,6 +219,15 @@ def pivot(self, pivot_col, values=None): jgd = self._jgd.pivot(pivot_col, values) return GroupedData(jgd, self._df) + @since(3.0) + def cogroup(self, other): + """ + Cogroups this group with another group so that we can run cogrouped operations. + + See :class:`CoGroupedData` for the operations that can be run. + """ + return CoGroupedData(self, other) + @since(2.3) def apply(self, udf): """ @@ -232,7 +242,7 @@ def apply(self, udf): The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the returnType of the pandas udf. - .. note:: This function requires a full shuffle. all the data of a group will be loaded + .. note:: This function requires a full shuffle. All the data of a group will be loaded into memory, so the user should be aware of the potential OOM risk if data is skewed and certain groups are too large to fit in memory. diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py new file mode 100644 index 0000000000000..7f3f7fa3168a7 --- /dev/null +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -0,0 +1,280 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import unittest +import sys + +from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType +from pyspark.sql.types import DoubleType, StructType, StructField +from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ + pandas_requirement_message, pyarrow_requirement_message +from pyspark.testing.utils import QuietTest + +if have_pandas: + import pandas as pd + from pandas.util.testing import assert_frame_equal, assert_series_equal + +if have_pyarrow: + import pyarrow as pa + + +""" +Tests below use pd.DataFrame.assign that will infer mixed types (unicode/str) for column names +from kwargs w/ Python 2, so need to set check_column_type=False and avoid this check +""" +if sys.version < '3': + _check_column_type = False +else: + _check_column_type = True + + +@unittest.skipIf( + not have_pandas or not have_pyarrow, + pandas_requirement_message or pyarrow_requirement_message) +class CoGroupedMapPandasUDFTests(ReusedSQLTestCase): + + @property + def data1(self): + return self.spark.range(10).toDF('id') \ + .withColumn("ks", array([lit(i) for i in range(20, 30)])) \ + .withColumn("k", explode(col('ks')))\ + .withColumn("v", col('k') * 10)\ + .drop('ks') + + @property + def data2(self): + return self.spark.range(10).toDF('id') \ + .withColumn("ks", array([lit(i) for i in range(20, 30)])) \ + .withColumn("k", explode(col('ks'))) \ + .withColumn("v2", col('k') * 100) \ + .drop('ks') + + def test_simple(self): + self._test_merge(self.data1, self.data2) + + def test_left_group_empty(self): + left = self.data1.where(col("id") % 2 == 0) + self._test_merge(left, self.data2) + + def test_right_group_empty(self): + right = self.data2.where(col("id") % 2 == 0) + self._test_merge(self.data1, right) + + def test_different_schemas(self): + right = self.data2.withColumn('v3', lit('a')) + self._test_merge(self.data1, right, 'id long, k int, v int, v2 int, v3 string') + + def test_complex_group_by(self): + left = pd.DataFrame.from_dict({ + 'id': [1, 2, 3], + 'k': [5, 6, 7], + 'v': [9, 10, 11] + }) + + right = pd.DataFrame.from_dict({ + 'id': [11, 12, 13], + 'k': [5, 6, 7], + 'v2': [90, 100, 110] + }) + + left_gdf = self.spark\ + .createDataFrame(left)\ + .groupby(col('id') % 2 == 0) + + right_gdf = self.spark \ + .createDataFrame(right) \ + .groupby(col('id') % 2 == 0) + + @pandas_udf('k long, v long, v2 long', PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l[['k', 'v']], r[['k', 'v2']], on=['k']) + + result = left_gdf \ + .cogroup(right_gdf) \ + .apply(merge_pandas) \ + .sort(['k']) \ + .toPandas() + + expected = pd.DataFrame.from_dict({ + 'k': [5, 6, 7], + 'v': [9, 10, 11], + 'v2': [90, 100, 110] + }) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_empty_group_by(self): + left = self.data1 + right = self.data2 + + @pandas_udf('id long, k int, v int, v2 int', PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l, r, on=['id', 'k']) + + result = left.groupby().cogroup(right.groupby())\ + .apply(merge_pandas) \ + .sort(['id', 'k']) \ + .toPandas() + + left = left.toPandas() + right = right.toPandas() + + expected = pd \ + .merge(left, right, on=['id', 'k']) \ + .sort_values(by=['id', 'k']) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_mixed_scalar_udfs_followed_by_cogrouby_apply(self): + df = self.spark.range(0, 10).toDF('v1') + df = df.withColumn('v2', udf(lambda x: x + 1, 'int')(df['v1'])) \ + .withColumn('v3', pandas_udf(lambda x: x + 2, 'int')(df['v1'])) + + result = df.groupby().cogroup(df.groupby())\ + .apply(pandas_udf(lambda x, y: pd.DataFrame([(x.sum().sum(), y.sum().sum())]), + 'sum1 int, sum2 int', + PandasUDFType.COGROUPED_MAP)).collect() + + self.assertEquals(result[0]['sum1'], 165) + self.assertEquals(result[0]['sum2'], 165) + + def test_with_key_left(self): + self._test_with_key(self.data1, self.data1, isLeft=True) + + def test_with_key_right(self): + self._test_with_key(self.data1, self.data1, isLeft=False) + + def test_with_key_left_group_empty(self): + left = self.data1.where(col("id") % 2 == 0) + self._test_with_key(left, self.data1, isLeft=True) + + def test_with_key_right_group_empty(self): + right = self.data1.where(col("id") % 2 == 0) + self._test_with_key(self.data1, right, isLeft=False) + + def test_with_key_complex(self): + + @pandas_udf('id long, k int, v int, key boolean', PandasUDFType.COGROUPED_MAP) + def left_assign_key(key, l, _): + return l.assign(key=key[0]) + + result = self.data1 \ + .groupby(col('id') % 2 == 0)\ + .cogroup(self.data2.groupby(col('id') % 2 == 0)) \ + .apply(left_assign_key) \ + .sort(['id', 'k']) \ + .toPandas() + + expected = self.data1.toPandas() + expected = expected.assign(key=expected.id % 2 == 0) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_wrong_return_type(self): + with QuietTest(self.sc): + with self.assertRaisesRegexp( + NotImplementedError, + 'Invalid returnType.*cogrouped map Pandas UDF.*MapType'): + pandas_udf( + lambda l, r: l, + 'id long, v map', + PandasUDFType.COGROUPED_MAP) + + def test_wrong_args(self): + # Test that we get a sensible exception invalid values passed to apply + left = self.data1 + right = self.data2 + with QuietTest(self.sc): + # Function rather than a udf + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(lambda l, r: l) + + # Udf missing return type + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(udf(lambda l, r: l, DoubleType())) + + # Pass in expression rather than udf + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(left.v + 1) + + # Zero arg function + with self.assertRaisesRegexp(ValueError, 'Invalid function'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) + + # Udf without PandasUDFType + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda x, y: x, DoubleType())) + + # Udf with incorrect PandasUDFType + with self.assertRaisesRegexp(ValueError, 'Invalid udf.*COGROUPED_MAP'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) + + @staticmethod + def _test_with_key(left, right, isLeft): + + @pandas_udf('id long, k int, v int, key long', PandasUDFType.COGROUPED_MAP) + def right_assign_key(key, l, r): + return l.assign(key=key[0]) if isLeft else r.assign(key=key[0]) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ + .apply(right_assign_key) \ + .toPandas() + + expected = left.toPandas() if isLeft else right.toPandas() + expected = expected.assign(key=expected.id) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + @staticmethod + def _test_merge(left, right, output_schema='id long, k int, v int, v2 int'): + + @pandas_udf(output_schema, PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l, r, on=['id', 'k']) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ + .apply(merge_pandas)\ + .sort(['id', 'k']) \ + .toPandas() + + left = left.toPandas() + right = right.toPandas() + + expected = pd \ + .merge(left, right, on=['id', 'k']) \ + .sort_values(by=['id', 'k']) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + +if __name__ == "__main__": + from pyspark.sql.tests.test_pandas_udf_cogrouped_map import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 188ec2634974a..c4d7c1ed205f1 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -42,6 +42,7 @@ def _create_udf(f, returnType, evalType): if evalType in (PythonEvalType.SQL_SCALAR_PANDAS_UDF, PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF): @@ -65,6 +66,13 @@ def _create_udf(f, returnType, evalType): "Invalid function: pandas_udfs with function type GROUPED_MAP " "must take either one argument (data) or two arguments (key, data).") + if evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF \ + and len(argspec.args) not in (2, 3): + raise ValueError( + "Invalid function: pandas_udfs with function type COGROUPED_MAP " + "must take either two arguments (left, right) " + "or three arguments (key, left, right).") + # Set the name of the UserDefinedFunction object to be the name of function f udf_obj = UserDefinedFunction( f, returnType=returnType, name=None, evalType=evalType, deterministic=True) @@ -147,6 +155,17 @@ def returnType(self): else: raise TypeError("Invalid returnType for map iterator Pandas " "UDFs: returnType must be a StructType.") + elif self.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + if isinstance(self._returnType_placeholder, StructType): + try: + to_arrow_type(self._returnType_placeholder) + except TypeError: + raise NotImplementedError( + "Invalid returnType with cogrouped map Pandas UDFs: " + "%s is not supported" % str(self._returnType_placeholder)) + else: + raise TypeError("Invalid returnType for cogrouped map Pandas " + "UDFs: returnType must be a StructType.") elif self.evalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: try: # StructType is not yet allowed as a return type, explicitly check here to fail fast diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 7f38c27360ed9..086202de2c68b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -39,7 +39,7 @@ from pyspark.rdd import PythonEvalType from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - BatchedSerializer, ArrowStreamPandasUDFSerializer + BatchedSerializer, ArrowStreamPandasUDFSerializer, CogroupUDFSerializer from pyspark.sql.types import to_arrow_type, StructType from pyspark.util import _get_argspec, fail_on_stopiteration from pyspark import shuffle @@ -121,6 +121,33 @@ def verify_result_type(result): map(verify_result_type, f(*iterator))) +def wrap_cogrouped_map_pandas_udf(f, return_type, argspec): + + def wrapped(left_key_series, left_value_series, right_key_series, right_value_series): + import pandas as pd + + left_df = pd.concat(left_value_series, axis=1) + right_df = pd.concat(right_value_series, axis=1) + + if len(argspec.args) == 2: + result = f(left_df, right_df) + elif len(argspec.args) == 3: + key_series = left_key_series if not left_df.empty else right_key_series + key = tuple(s[0] for s in key_series) + result = f(key, left_df, right_df) + if not isinstance(result, pd.DataFrame): + raise TypeError("Return type of the user-defined function should be " + "pandas.DataFrame, but is {}".format(type(result))) + if not len(result.columns) == len(return_type): + raise RuntimeError( + "Number of columns of the returned pandas.DataFrame " + "doesn't match specified schema. " + "Expected: {} Actual: {}".format(len(return_type), len(result.columns))) + return result + + return lambda kl, vl, kr, vr: [(wrapped(kl, vl, kr, vr), to_arrow_type(return_type))] + + def wrap_grouped_map_pandas_udf(f, return_type, argspec): def wrapped(key_series, value_series): @@ -244,6 +271,9 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: argspec = _get_argspec(chained_func) # signature was lost when wrapping it return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec) + elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + argspec = _get_argspec(chained_func) # signature was lost when wrapping it + return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF: @@ -258,6 +288,7 @@ def read_udfs(pickleSer, infile, eval_type): runner_conf = {} if eval_type in (PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, @@ -280,13 +311,16 @@ def read_udfs(pickleSer, infile, eval_type): "spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName", "true")\ .lower() == "true" - # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of - # pandas Series. See SPARK-27240. - df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or - eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or - eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) - ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name, - df_for_struct) + if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + ser = CogroupUDFSerializer(timezone, safecheck, assign_cols_by_name) + else: + # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of + # pandas Series. See SPARK-27240. + df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or + eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or + eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) + ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name, + df_for_struct) else: ser = BatchedSerializer(PickleSerializer(), 100) @@ -343,6 +377,32 @@ def map_batch(batch): # profiling is not supported for UDF return func, None, ser, ser + def extract_key_value_indexes(grouped_arg_offsets): + """ + Helper function to extract the key and value indexes from arg_offsets for the grouped and + cogrouped pandas udfs. See BasePandasGroupExec.resolveArgOffsets for equivalent scala code. + + :param grouped_arg_offsets: List containing the key and value indexes of columns of the + DataFrames to be passed to the udf. It consists of n repeating groups where n is the + number of DataFrames. Each group has the following format: + group[0]: length of group + group[1]: length of key indexes + group[2.. group[1] +2]: key attributes + group[group[1] +3 group[0]]: value attributes + """ + parsed = [] + idx = 0 + while idx < len(grouped_arg_offsets): + offsets_len = grouped_arg_offsets[idx] + idx += 1 + offsets = grouped_arg_offsets[idx: idx + offsets_len] + split_index = offsets[0] + 1 + offset_keys = offsets[1: split_index] + offset_values = offsets[split_index:] + parsed.append([offset_keys, offset_values]) + idx += offsets_len + return parsed + udfs = {} call_udf = [] mapper_str = "" @@ -359,10 +419,24 @@ def map_batch(batch): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - split_offset = arg_offsets[0] + 1 - arg0 = ["a[%d]" % o for o in arg_offsets[1: split_offset]] - arg1 = ["a[%d]" % o for o in arg_offsets[split_offset:]] - mapper_str = "lambda a: f([%s], [%s])" % (", ".join(arg0), ", ".join(arg1)) + parsed_offsets = extract_key_value_indexes(arg_offsets) + keys = ["a[%d]" % (o,) for o in parsed_offsets[0][0]] + vals = ["a[%d]" % (o, ) for o in parsed_offsets[0][1]] + mapper_str = "lambda a: f([%s], [%s])" % (", ".join(keys), ", ".join(vals)) + elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + # We assume there is only one UDF here because cogrouped map doesn't + # support combining multiple UDFs. + assert num_udfs == 1 + arg_offsets, udf = read_single_udf( + pickleSer, infile, eval_type, runner_conf, udf_index=0) + udfs['f'] = udf + parsed_offsets = extract_key_value_indexes(arg_offsets) + df1_keys = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][0]] + df1_vals = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][1]] + df2_keys = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][0]] + df2_vals = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][1]] + mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % ( + ", ".join(df1_keys), ", ".join(df1_vals), ", ".join(df2_keys), ", ".join(df2_vals)) else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) 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 413c75d2581d1..0a13a34720846 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 @@ -1191,6 +1191,12 @@ class Analyzer( // To resolve duplicate expression IDs for Join and Intersect case j @ Join(left, right, _, _, _) if !j.duplicateResolved => j.copy(right = dedupRight(left, right)) + case f @ FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, _, _, left, right) => + val leftRes = leftAttributes + .map(x => resolveExpressionBottomUp(x, left).asInstanceOf[Attribute]) + val rightRes = rightAttributes + .map(x => resolveExpressionBottomUp(x, right).asInstanceOf[Attribute]) + f.copy(leftAttributes = leftRes, rightAttributes = rightRes) // intersect/except will be rewritten to join at the begininng of optimizer. Here we need to // deduplicate the right side plan, so that we won't produce an invalid self-join later. case i @ Intersect(left, right, _) if !i.duplicateResolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index dc2185194d84e..c4f741cd2cec8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF} /** - * FlatMap groups using an udf: pandas.Dataframe -> pandas.DataFrame. + * FlatMap groups using a udf: pandas.Dataframe -> pandas.DataFrame. * This is used by DataFrame.groupby().apply(). */ case class FlatMapGroupsInPandas( @@ -40,7 +40,7 @@ case class FlatMapGroupsInPandas( } /** - * Map partitions using an udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). + * Map partitions using a udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). * This is used by DataFrame.mapInPandas() */ case class MapInPandas( @@ -51,6 +51,21 @@ case class MapInPandas( override val producedAttributes = AttributeSet(output) } +/** + * Flatmap cogroups using a udf: pandas.Dataframe, pandas.Dataframe -> pandas.Dataframe + * This is used by DataFrame.groupby().cogroup().apply(). + */ +case class FlatMapCoGroupsInPandas( + leftAttributes: Seq[Attribute], + rightAttributes: Seq[Attribute], + functionExpr: Expression, + output: Seq[Attribute], + left: LogicalPlan, + right: LogicalPlan) extends BinaryNode { + + override val producedAttributes = AttributeSet(output) +} + trait BaseEvalPython extends UnaryNode { def udfs: Seq[PythonUDF] 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 e85636d82a62c..f6d13be0e89be 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]( - df: DataFrame, - groupingExprs: Seq[Expression], + val df: DataFrame, + val groupingExprs: Seq[Expression], groupType: RelationalGroupedDataset.GroupType) { private[this] def toDF(aggExprs: Seq[Expression]): DataFrame = { @@ -523,6 +523,48 @@ class RelationalGroupedDataset protected[sql]( Dataset.ofRows(df.sparkSession, plan) } + /** + * Applies a vectorized python user-defined function to each cogrouped data. + * The user-defined function defines a transformation: + * `pandas.DataFrame`, `pandas.DataFrame` -> `pandas.DataFrame`. + * For each group in the cogrouped data, all elements in the group are passed as a + * `pandas.DataFrame` and the results for all cogroups are combined into a new [[DataFrame]]. + * + * This function uses Apache Arrow as serialization format between Java executors and Python + * workers. + */ + private[sql] def flatMapCoGroupsInPandas( + r: RelationalGroupedDataset, + expr: PythonUDF): DataFrame = { + require(expr.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + "Must pass a cogrouped map udf") + require(expr.dataType.isInstanceOf[StructType], + s"The returnType of the udf must be a ${StructType.simpleString}") + + val leftGroupingNamedExpressions = groupingExprs.map { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + + val rightGroupingNamedExpressions = r.groupingExprs.map { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + + val leftAttributes = leftGroupingNamedExpressions.map(_.toAttribute) + val rightAttributes = rightGroupingNamedExpressions.map(_.toAttribute) + + val leftChild = df.logicalPlan + val rightChild = r.df.logicalPlan + + val left = Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild) + val right = Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild) + + val output = expr.dataType.asInstanceOf[StructType].toAttributes + val plan = FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, expr, output, left, right) + Dataset.ofRows(df.sparkSession, plan) + } + override def toString: String = { val builder = new StringBuilder builder.append("RelationalGroupedDataset: [grouping expressions: [") 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 85469bf2401d4..a2f45898d273f 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 @@ -682,6 +682,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { f, p, b, is, ot, planLater(child)) :: Nil case logical.FlatMapGroupsInPandas(grouping, func, output, child) => execution.python.FlatMapGroupsInPandasExec(grouping, func, output, planLater(child)) :: Nil + case logical.FlatMapCoGroupsInPandas(leftGroup, rightGroup, func, output, left, right) => + execution.python.FlatMapCoGroupsInPandasExec( + leftGroup, rightGroup, func, output, planLater(left), planLater(right)) :: Nil case logical.MapInPandas(func, output, child) => execution.python.MapInPandasExec(func, output, planLater(child)) :: Nil case logical.MapElements(f, _, _, objAttr, child) => 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 5101f7e871af2..fcf68467460bb 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 @@ -19,12 +19,9 @@ package org.apache.spark.sql.execution.python import java.io._ import java.net._ -import java.util.concurrent.atomic.AtomicBoolean - -import scala.collection.JavaConverters._ import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} +import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.spark._ import org.apache.spark.api.python._ @@ -33,7 +30,6 @@ import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.util.Utils /** @@ -46,7 +42,7 @@ class ArrowPythonRunner( schema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch]( + extends BaseArrowPythonRunner[Iterator[InternalRow]]( funcs, evalType, argOffsets) { override val bufferSize: Int = SQLConf.get.pandasUDFBufferSize @@ -119,72 +115,4 @@ class ArrowPythonRunner( } } - protected override def newReaderIterator( - stream: DataInputStream, - writerThread: WriterThread, - startTime: Long, - env: SparkEnv, - worker: Socket, - releasedOrClosed: AtomicBoolean, - context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { - - private val allocator = ArrowUtils.rootAllocator.newChildAllocator( - s"stdin reader for $pythonExec", 0, Long.MaxValue) - - private var reader: ArrowStreamReader = _ - private var root: VectorSchemaRoot = _ - private var schema: StructType = _ - private var vectors: Array[ColumnVector] = _ - - context.addTaskCompletionListener[Unit] { _ => - if (reader != null) { - reader.close(false) - } - allocator.close() - } - - private var batchLoaded = true - - protected override def read(): ColumnarBatch = { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - try { - if (reader != null && batchLoaded) { - batchLoaded = reader.loadNextBatch() - if (batchLoaded) { - val batch = new ColumnarBatch(vectors) - batch.setNumRows(root.getRowCount) - batch - } else { - reader.close(false) - allocator.close() - // Reach end of stream. Call `read()` again to read control data. - read() - } - } else { - stream.readInt() match { - case SpecialLengths.START_ARROW_STREAM => - reader = new ArrowStreamReader(stream, allocator) - root = reader.getVectorSchemaRoot() - schema = ArrowUtils.fromArrowSchema(root.getSchema()) - vectors = root.getFieldVectors().asScala.map { vector => - new ArrowColumnVector(vector) - }.toArray[ColumnVector] - read() - case SpecialLengths.TIMING_DATA => - handleTimingData() - read() - case SpecialLengths.PYTHON_EXCEPTION_THROWN => - throw handlePythonException() - case SpecialLengths.END_OF_DATA_SECTION => - handleEndOfDataSection() - null - } - } - } catch handleException - } - } - } } 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/BaseArrowPythonRunner.scala new file mode 100644 index 0000000000000..0cee7d2f96c22 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import java.io._ +import java.net._ +import java.util.concurrent.atomic.AtomicBoolean + +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.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. + */ +abstract class BaseArrowPythonRunner[T]( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]]) + extends BasePythonRunner[T, ColumnarBatch](funcs, evalType, argOffsets) { + + protected override def newReaderIterator( + stream: DataInputStream, + writerThread: WriterThread, + startTime: Long, + env: SparkEnv, + worker: Socket, + releasedOrClosed: AtomicBoolean, + context: TaskContext): Iterator[ColumnarBatch] = { + + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + + private val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdin reader for $pythonExec", 0, Long.MaxValue) + + private var reader: ArrowStreamReader = _ + private var root: VectorSchemaRoot = _ + private var schema: StructType = _ + private var vectors: Array[ColumnVector] = _ + + context.addTaskCompletionListener[Unit] { _ => + if (reader != null) { + reader.close(false) + } + allocator.close() + } + + private var batchLoaded = true + + protected override def read(): ColumnarBatch = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + try { + if (reader != null && batchLoaded) { + batchLoaded = reader.loadNextBatch() + if (batchLoaded) { + val batch = new ColumnarBatch(vectors) + batch.setNumRows(root.getRowCount) + batch + } else { + reader.close(false) + allocator.close() + // Reach end of stream. Call `read()` again to read control data. + read() + } + } else { + stream.readInt() match { + case SpecialLengths.START_ARROW_STREAM => + reader = new ArrowStreamReader(stream, allocator) + root = reader.getVectorSchemaRoot() + schema = ArrowUtils.fromArrowSchema(root.getSchema()) + vectors = root.getFieldVectors().asScala.map { vector => + new ArrowColumnVector(vector) + }.toArray[ColumnVector] + read() + case SpecialLengths.TIMING_DATA => + handleTimingData() + read() + case SpecialLengths.PYTHON_EXCEPTION_THROWN => + throw handlePythonException() + case SpecialLengths.END_OF_DATA_SECTION => + handleEndOfDataSection() + null + } + } + } catch handleException + } + } + } +} 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/BasePandasGroupExec.scala new file mode 100644 index 0000000000000..477c288ad1211 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +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.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF, 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) + + /** + * passes the data to the python runner and coverts the resulting + * columnarbatch into internal rows. + */ + protected def executePython[T]( + data: Iterator[T], + runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { + + val context = TaskContext.get() + val columnarBatchIter = runner.compute(data, context.partitionId(), context) + val unsafeProj = UnsafeProjection.create(output, output) + + columnarBatchIter.flatMap { batch => + // UDF returns a StructType column in ColumnarBatch, select the children here + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] + val outputVectors = output.indices.map(structVector.getChild) + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) + flattenedBatch.setNumRows(batch.numRows()) + flattenedBatch.rowIterator.asScala + }.map(unsafeProj) + } + + /** + * groups according to grouping attributes and then projects into the deduplicated schema + */ + protected def groupAndProject( + input: Iterator[InternalRow], + groupingAttributes: Seq[Attribute], + inputSchema: Seq[Attribute], + dedupSchema: Seq[Attribute]): Iterator[(InternalRow, Iterator[InternalRow])] = { + val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) + val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) + groupedIter.map { + case (k, groupedRowIter) => (k, groupedRowIter.map(dedupProj)) + } + } + + /** + * Returns a the deduplicated attributes of the spark plan and the arg offsets of the + * keys and values. + * + * The deduplicated attributes are needed because the spark plan may contain an attribute + * twice; once in the key and once in the value. For any such attribute we need to + * deduplicate. + * + * The arg offsets are used to distinguish grouping grouping attributes and data attributes + * as following: + * + * argOffsets[0] is the length of the argOffsets array + * + * argOffsets[1] is the length of grouping attribute + * argOffsets[2 .. argOffsets[0]+2] is the arg offsets for grouping attributes + * + * argOffsets[argOffsets[0]+2 .. ] is the arg offsets for data attributes + */ + protected def resolveArgOffsets( + child: SparkPlan, groupingAttributes: Seq[Attribute]): (Seq[Attribute], Array[Int]) = { + + val dataAttributes = child.output.drop(groupingAttributes.length) + val groupingIndicesInData = groupingAttributes.map { attribute => + dataAttributes.indexWhere(attribute.semanticEquals) + } + + val groupingArgOffsets = new ArrayBuffer[Int] + val nonDupGroupingAttributes = new ArrayBuffer[Attribute] + val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) + + groupingAttributes.zip(groupingIndicesInData).foreach { + case (attribute, index) => + if (index == -1) { + groupingArgOffsets += nonDupGroupingAttributes.length + nonDupGroupingAttributes += attribute + } else { + groupingArgOffsets += index + nonDupGroupingSize + } + } + + val dataArgOffsets = nonDupGroupingAttributes.length until + (nonDupGroupingAttributes.length + dataAttributes.length) + + val argOffsetsLength = groupingAttributes.length + dataArgOffsets.length + 1 + val argOffsets = Array(argOffsetsLength, + groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets + + // Attributes after deduplication + val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes + (dedupAttributes, argOffsets) + } +} 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 new file mode 100644 index 0000000000000..8ea9881c575a1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import java.io._ +import java.net._ + +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.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.arrow.ArrowWriter +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils +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. + */ +class CogroupedArrowPythonRunner( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]], + leftSchema: StructType, + rightSchema: StructType, + timeZoneId: String, + conf: Map[String, String]) + extends BaseArrowPythonRunner[(Iterator[InternalRow], Iterator[InternalRow])]( + funcs, evalType, argOffsets) { + + protected def newWriterThread( + env: SparkEnv, + worker: Socket, + inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], + partitionIndex: Int, + context: TaskContext): WriterThread = { + + new WriterThread(env, worker, inputIterator, partitionIndex, context) { + + protected override def writeCommand(dataOut: DataOutputStream): Unit = { + + // Write config for the worker as a number of key -> value pairs of strings + dataOut.writeInt(conf.size) + for ((k, v) <- conf) { + PythonRDD.writeUTF(k, dataOut) + PythonRDD.writeUTF(v, dataOut) + } + + PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + } + + protected override def writeIteratorToStream(dataOut: DataOutputStream): Unit = { + // For each we first send the number of dataframes in each group then send + // first df, then send second df. End of data is marked by sending 0. + while (inputIterator.hasNext) { + dataOut.writeInt(2) + val (nextLeft, nextRight) = inputIterator.next() + writeGroup(nextLeft, leftSchema, dataOut, "left") + writeGroup(nextRight, rightSchema, dataOut, "right") + } + dataOut.writeInt(0) + } + + def writeGroup( + group: Iterator[InternalRow], + schema: StructType, + dataOut: DataOutputStream, + name: String) = { + val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) + val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdout writer for $pythonExec ($name)", 0, Long.MaxValue) + val root = VectorSchemaRoot.create(arrowSchema, allocator) + + Utils.tryWithSafeFinally { + val writer = new ArrowStreamWriter(root, null, dataOut) + val arrowWriter = ArrowWriter.create(root) + writer.start() + + while (group.hasNext) { + arrowWriter.write(group.next()) + } + arrowWriter.finish() + writer.writeBatch() + writer.end() + }{ + root.close() + allocator.close() + } + } + } + } +} + 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 new file mode 100644 index 0000000000000..cc83e0cecdc33 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import org.apache.spark.api.python.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.types.StructType + + +/** + * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapCoGroupsInPandas]] + * + * The input dataframes are first Cogrouped. Rows from each side of the cogroup are passed to the + * Python worker via Arrow. As each side of the cogroup may have a different schema we send every + * group in its own Arrow stream. + * The Python worker turns the resulting record batches to `pandas.DataFrame`s, invokes the + * user-defined function, and passes the resulting `pandas.DataFrame` + * as an Arrow record batch. Finally, each record batch is turned to + * Iterator[InternalRow] using ColumnarBatch. + * + * Note on memory usage: + * Both the Python worker and the Java executor need to have enough memory to + * hold the largest cogroup. The memory on the Java side is used to construct the + * record batches (off heap memory). The memory on the Python side is used for + * holding the `pandas.DataFrame`. It's possible to further split one group into + * multiple record batches to reduce the memory footprint on the Java side, this + * is left as future work. + */ +case class FlatMapCoGroupsInPandasExec( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan) + extends BasePandasGroupExec(func, output) with BinaryExecNode { + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = { + val leftDist = if (leftGroup.isEmpty) AllTuples else ClusteredDistribution(leftGroup) + val rightDist = if (rightGroup.isEmpty) AllTuples else ClusteredDistribution(rightGroup) + leftDist :: rightDist :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + leftGroup + .map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil + } + + override protected def doExecute(): RDD[InternalRow] = { + + val (leftDedup, leftArgOffsets) = resolveArgOffsets(left, leftGroup) + val (rightDedup, rightArgOffsets) = resolveArgOffsets(right, rightGroup) + + // Map cogrouped rows to ArrowPythonRunner results, Only execute if partition is not empty + left.execute().zipPartitions(right.execute()) { (leftData, rightData) => + if (leftData.isEmpty && rightData.isEmpty) Iterator.empty else { + + val leftGrouped = groupAndProject(leftData, leftGroup, left.output, leftDedup) + val rightGrouped = groupAndProject(rightData, rightGroup, right.output, rightDedup) + val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) + .map { case (_, l, r) => (l, r) } + + val runner = new CogroupedArrowPythonRunner( + chainedFunc, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + Array(leftArgOffsets ++ rightArgOffsets), + StructType.fromAttributes(leftDedup), + StructType.fromAttributes(rightDedup), + sessionLocalTimeZone, + pythonRunnerConf) + + executePython(data, 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 267698d1bca50..22a0d1e09b12e 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,19 +17,14 @@ package org.apache.spark.sql.execution.python -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.TaskContext -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.api.python.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.{GroupedIterator, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + /** * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsInPandas]] @@ -53,14 +48,10 @@ case class FlatMapGroupsInPandasExec( func: Expression, output: Seq[Attribute], child: SparkPlan) - extends UnaryExecNode { - - private val pandasFunction = func.asInstanceOf[PythonUDF].func + extends BasePandasGroupExec(func, output) with UnaryExecNode { override def outputPartitioning: Partitioning = child.outputPartitioning - override def producedAttributes: AttributeSet = AttributeSet(output) - override def requiredChildDistribution: Seq[Distribution] = { if (groupingAttributes.isEmpty) { AllTuples :: Nil @@ -75,88 +66,23 @@ case class FlatMapGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { val inputRDD = child.execute() - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) - - // Deduplicate the grouping attributes. - // If a grouping attribute also appears in data attributes, then we don't need to send the - // grouping attribute to Python worker. If a grouping attribute is not in data attributes, - // then we need to send this grouping attribute to python worker. - // - // We use argOffsets to distinguish grouping attributes and data attributes as following: - // - // argOffsets[0] is the length of grouping attributes - // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes - // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes - - val dataAttributes = child.output.drop(groupingAttributes.length) - val groupingIndicesInData = groupingAttributes.map { attribute => - dataAttributes.indexWhere(attribute.semanticEquals) - } - - val groupingArgOffsets = new ArrayBuffer[Int] - val nonDupGroupingAttributes = new ArrayBuffer[Attribute] - val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) - - // Non duplicate grouping attributes are added to nonDupGroupingAttributes and - // their offsets are 0, 1, 2 ... - // Duplicate grouping attributes are NOT added to nonDupGroupingAttributes and - // their offsets are n + index, where n is the total number of non duplicate grouping - // attributes and index is the index in the data attributes that the grouping attribute - // is a duplicate of. - - groupingAttributes.zip(groupingIndicesInData).foreach { - case (attribute, index) => - if (index == -1) { - groupingArgOffsets += nonDupGroupingAttributes.length - nonDupGroupingAttributes += attribute - } else { - groupingArgOffsets += index + nonDupGroupingSize - } - } - - val dataArgOffsets = nonDupGroupingAttributes.length until - (nonDupGroupingAttributes.length + dataAttributes.length) - - val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) - - // Attributes after deduplication - val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes - val dedupSchema = StructType.fromAttributes(dedupAttributes) + val (dedupAttributes, argOffsets) = resolveArgOffsets(child, groupingAttributes) // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { - val grouped = if (groupingAttributes.isEmpty) { - Iterator(iter) - } else { - val groupedIter = GroupedIterator(iter, groupingAttributes, child.output) - val dedupProj = UnsafeProjection.create(dedupAttributes, child.output) - groupedIter.map { - case (_, groupedRowIter) => groupedRowIter.map(dedupProj) - } - } - val context = TaskContext.get() + val data = groupAndProject(iter, groupingAttributes, child.output, dedupAttributes) + .map{case(_, x) => x} - val columnarBatchIter = new ArrowPythonRunner( + val runner = new ArrowPythonRunner( chainedFunc, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, - argOffsets, - dedupSchema, + Array(argOffsets), + StructType.fromAttributes(dedupAttributes), sessionLocalTimeZone, - pythonRunnerConf).compute(grouped, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) + pythonRunnerConf) - columnarBatchIter.flatMap { batch => - // Grouped Map UDF returns a StructType column in ColumnarBatch, select the children here - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) + executePython(data, runner) }} } } From 3ece8ee15775307bded572ac391aeed10be3c9aa Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 Sep 2019 10:33:03 +0900 Subject: [PATCH 27/87] [SPARK-29124][CORE] Use MurmurHash3 `bytesHash(data, seed)` instead of `bytesHash(data)` ### What changes were proposed in this pull request? This PR changes `bytesHash(data)` API invocation with the underlaying `byteHash(data, arraySeed)` invocation. ```scala def bytesHash(data: Array[Byte]): Int = bytesHash(data, arraySeed) ``` ### Why are the changes needed? The original API is changed between Scala versions by the following commit. From Scala 2.12.9, the semantic of the function is changed. If we use the underlying form, we are safe during Scala version migration. - https://github.com/scala/scala/commit/846ee2b1a47014c69ebd2352d91d467be74918b5#diff-ac889f851e109fc4387cd738d52ce177 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This is a kind of refactoring. Pass the Jenkins with the existing tests. Closes #25821 from dongjoon-hyun/SPARK-SCALA-HASH. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/util/random/XORShiftRandom.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index af09e50a157ae..11aa9da83b9ed 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -60,7 +60,7 @@ private[spark] object XORShiftRandom { /** Hash seeds to have 0/1 bits throughout. */ private[random] def hashSeed(seed: Long): Long = { val bytes = ByteBuffer.allocate(java.lang.Long.BYTES).putLong(seed).array() - val lowBits = MurmurHash3.bytesHash(bytes) + val lowBits = MurmurHash3.bytesHash(bytes, MurmurHash3.arraySeed) val highBits = MurmurHash3.bytesHash(bytes, lowBits) (highBits.toLong << 32) | (lowBits.toLong & 0xFFFFFFFFL) } From 4559a82a1de289093064490ef2d39c3c535fb3d4 Mon Sep 17 00:00:00 2001 From: s71955 Date: Wed, 18 Sep 2019 12:54:44 +0900 Subject: [PATCH 28/87] [SPARK-28930][SQL] Last Access Time value shall display 'UNKNOWN' in all clients **What changes were proposed in this pull request?** Issue 1 : modifications not required as these are different formats for the same info. In the case of a Spark DataFrame, null is correct. Issue 2 mentioned in JIRA Spark SQL "desc formatted tablename" is not showing the header # col_name,data_type,comment , seems to be the header has been removed knowingly as part of SPARK-20954. Issue 3: Corrected the Last Access time, the value shall display 'UNKNOWN' as currently system wont support the last access time evaluation, since hive was setting Last access time as '0' in metastore even though spark CatalogTable last access time value set as -1. this will make the validation logic of LasAccessTime where spark sets 'UNKNOWN' value if last access time value set as -1 (means not evaluated). **Does this PR introduce any user-facing change?** No **How was this patch tested?** Locally and corrected a ut. Attaching the test report below ![SPARK-28930](https://user-images.githubusercontent.com/12999161/64484908-83a1d980-d236-11e9-8062-9facf3003e5e.PNG) Closes #25720 from sujith71955/master_describe_info. Authored-by: s71955 Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/catalyst/catalog/interface.scala | 7 +++++-- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index ce8c23ac6dceb..01b21feab0dd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -117,7 +117,7 @@ case class CatalogTablePartition( } map.put("Created Time", new Date(createTime).toString) val lastAccess = { - if (-1 == lastAccessTime) "UNKNOWN" else new Date(lastAccessTime).toString + if (lastAccessTime <= 0) "UNKNOWN" else new Date(lastAccessTime).toString } map.put("Last Access", lastAccess) stats.foreach(s => map.put("Partition Statistics", s.simpleString)) @@ -320,12 +320,15 @@ case class CatalogTable( val map = new mutable.LinkedHashMap[String, String]() val tableProperties = properties.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") val partitionColumns = partitionColumnNames.map(quoteIdentifier).mkString("[", ", ", "]") + val lastAccess = { + if (lastAccessTime <= 0) "UNKNOWN" else new Date(lastAccessTime).toString + } identifier.database.foreach(map.put("Database", _)) map.put("Table", identifier.table) if (owner != null && owner.nonEmpty) map.put("Owner", owner) map.put("Created Time", new Date(createTime).toString) - map.put("Last Access", new Date(lastAccessTime).toString) + map.put("Last Access", lastAccess) map.put("Created By", "Spark " + createVersion) map.put("Type", tableType.name) provider.foreach(map.put("Provider", _)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index c6e4407c80e83..d63d741943bc7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2435,7 +2435,7 @@ class HiveDDLSuite .select("data_type") // check if the last access time doesnt have the default date of year // 1970 as its a wrong access time - assert(!(desc.first.toString.contains("1970"))) + assert((desc.first.toString.contains("UNKNOWN"))) } } From eef5e6d348e7025f48e8f69026254e07efe80894 Mon Sep 17 00:00:00 2001 From: turbofei Date: Wed, 18 Sep 2019 13:12:18 +0900 Subject: [PATCH 29/87] [SPARK-29113][DOC] Fix some annotation errors and remove meaningless annotations in project ### What changes were proposed in this pull request? In this PR, I fix some annotation errors and remove meaningless annotations in project. ### Why are the changes needed? There are some annotation errors and meaningless annotations in project. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Verified manually. Closes #25809 from turboFei/SPARK-29113. Authored-by: turbofei Signed-off-by: HyukjinKwon --- .../org/apache/spark/io/NioBufferedFileInputStream.java | 1 - .../main/java/org/apache/spark/memory/MemoryConsumer.java | 1 - .../apache/spark/shuffle/sort/ShuffleExternalSorter.java | 1 - .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 2 -- .../org/apache/spark/deploy/history/ApplicationCache.scala | 7 +++---- .../org/apache/spark/scheduler/TaskSetBlacklist.scala | 1 - .../main/scala/org/apache/spark/storage/BlockManager.scala | 1 - .../scheduler/cluster/mesos/MesosSchedulerUtils.scala | 1 - .../org/apache/spark/sql/execution/ExplainUtils.scala | 4 ++-- 9 files changed, 5 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java index 92bf0ecc1b5cb..a1e29a8c873da 100644 --- a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java +++ b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java @@ -51,7 +51,6 @@ public NioBufferedFileInputStream(File file) throws IOException { /** * Checks weather data is left to be read from the input stream. * @return true if data is left, false otherwise - * @throws IOException */ private boolean refill() throws IOException { if (!byteBuffer.hasRemaining()) { 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 4bfd2d358f36f..7b68b399e6e84 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -78,7 +78,6 @@ public void spill() throws IOException { * @param size the amount of memory should be released * @param trigger the MemoryConsumer that trigger this spilling * @return the amount of released memory in bytes - * @throws IOException */ public abstract long spill(long size, MemoryConsumer trigger) throws IOException; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 024756087bf7f..833744f4777ce 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -423,7 +423,6 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p * * @return metadata for the spill files written by this sorter. If no records were ever inserted * into this sorter, then this will return an empty array. - * @throws IOException */ public SpillInfo[] closeAndGetSpills() throws IOException { if (inMemSorter != null) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 1b206c11d9a8e..55e4e609c3c7b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -447,8 +447,6 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, /** * Merges another UnsafeExternalSorters into this one, the other one will be emptied. - * - * @throws IOException */ public void merge(UnsafeExternalSorter other) throws IOException { other.spill(); diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index 8c63fa65b40fd..fb2a67c2ab103 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -209,9 +209,8 @@ private[history] class ApplicationCache( /** * Register a filter for the web UI which checks for updates to the given app/attempt - * @param ui Spark UI to attach filters to - * @param appId application ID - * @param attemptId attempt ID + * @param key consisted of appId and attemptId + * @param loadedUI Spark UI to attach filters to */ private def registerFilter(key: CacheKey, loadedUI: LoadedAppUI): Unit = { require(loadedUI != null) @@ -231,7 +230,7 @@ private[history] class ApplicationCache( /** * An entry in the cache. * - * @param ui Spark UI + * @param loadedUI Spark UI * @param completed Flag to indicated that the application has completed (and so * does not need refreshing). */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index b680979a466a5..4df2889089ee9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -69,7 +69,6 @@ private[scheduler] class TaskSetBlacklist( /** * Get the most recent failure reason of this TaskSet. - * @return */ def getLatestFailureReason: String = { latestFailureReason diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4b71dc1fff345..0dfadb657b770 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -853,7 +853,6 @@ private[spark] class BlockManager( * @param bufferTransformer this transformer expected to open the file if the block is backed by a * file by this it is guaranteed the whole content can be loaded * @tparam T result type - * @return */ private[spark] def getRemoteBlock[T]( blockId: BlockId, 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 06993712035ff..f60b3b8db194c 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 @@ -285,7 +285,6 @@ trait MesosSchedulerUtils extends Logging { * The attribute values are the mesos attribute types and they are * * @param offerAttributes the attributes offered - * @return */ protected def toAttributeMap(offerAttributes: JList[Attribute]) : Map[String, GeneratedMessageV3] = { 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 18a7f9822dcbc..fc384fe117caf 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 @@ -41,7 +41,7 @@ object ExplainUtils { * * @param plan Input query plan to process * @param append function used to append the explain output - * @param startOperationID The start value of operation id. The subsequent operations will + * @param startOperatorID The start value of operation id. The subsequent operations will * be assigned higher value. * * @return The last generated operation id for this input plan. This is to ensure we @@ -125,7 +125,7 @@ object ExplainUtils { * appear in the explain output. * 2. operator identifier starts at startOperatorID + 1 * @param plan Input query plan to process - * @param startOperationID The start value of operation id. The subsequent operations will + * @param startOperatorID The start value of operation id. The subsequent operations will * be assigned higher value. * @param operatorIDs A output parameter that contains a map of operator id and query plan. This * is used by caller to print the detail portion of the plan. From 0b6775e6e9aea091e16d885639b16768650c513c Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 18 Sep 2019 14:11:39 +0800 Subject: [PATCH 30/87] [SPARK-29112][YARN] Expose more details when ApplicationMaster reporter faces a fatal exception ### What changes were proposed in this pull request? In `ApplicationMaster.Reporter` thread, fatal exception information is swallowed. It's better to expose it. We found our thrift server was shutdown due to a fatal exception but no useful information from log. > 19/09/16 06:59:54,498 INFO [Reporter] yarn.ApplicationMaster:54 : Final app status: FAILED, exitCode: 12, (reason: Exception was thrown 1 time(s) from Reporter thread.) 19/09/16 06:59:54,500 ERROR [Driver] thriftserver.HiveThriftServer2:91 : Error starting HiveThriftServer2 java.lang.InterruptedException: sleep interrupted at java.lang.Thread.sleep(Native Method) at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:160) at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$4.run(ApplicationMaster.scala:708) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual test Closes #25810 from LantaoJin/SPARK-29112. Authored-by: LantaoJin Signed-off-by: jerryshao --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 4d884d54501ad..68035e4321e01 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 @@ -578,7 +578,11 @@ private[spark] class ApplicationMaster( e.getMessage) case e: Throwable => failureCount += 1 - if (!NonFatal(e) || failureCount >= reporterMaxFailures) { + if (!NonFatal(e)) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_REPORTER_FAILURE, + "Fatal exception: " + StringUtils.stringifyException(e)) + } else if (failureCount >= reporterMaxFailures) { finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + s"$failureCount time(s) from Reporter thread.") From 3da2786dc6910fd1fd71541b191ca29da2f17f96 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 18 Sep 2019 16:59:17 +0800 Subject: [PATCH 31/87] [SPARK-29096][SQL] The exact math method should be called only when there is a corresponding function in Math ### What changes were proposed in this pull request? 1. After https://github.com/apache/spark/pull/21599, if the option "spark.sql.failOnIntegralTypeOverflow" is enabled, all the Binary Arithmetic operator will used the exact version function. However, only `Add`/`Substract`/`Multiply` has a corresponding exact function in java.lang.Math . When the option "spark.sql.failOnIntegralTypeOverflow" is enabled, a runtime exception "BinaryArithmetics must override either exactMathMethod or genCode" is thrown if the other Binary Arithmetic operators are used, such as "Divide", "Remainder". The exact math method should be called only when there is a corresponding function in `java.lang.Math` 2. Revise the log output of casting to `Int`/`Short` 3. Enable `spark.sql.failOnIntegralTypeOverflow` for pgSQL tests in `SQLQueryTestSuite`. ### Why are the changes needed? 1. Fix the bugs of https://github.com/apache/spark/pull/21599 2. The test case of pgSQL intends to check the overflow of integer/long type. We should enable `spark.sql.failOnIntegralTypeOverflow`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test. Closes #25804 from gengliangwang/enableIntegerOverflowInSQLTest. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Cast.scala | 18 +- .../sql/catalyst/expressions/arithmetic.scala | 17 +- .../org/apache/spark/sql/types/Decimal.scala | 2 +- .../org/apache/spark/sql/types/numerics.scala | 6 +- .../resources/sql-tests/inputs/pgSQL/int4.sql | 5 - .../sql-tests/results/pgSQL/float4.sql.out | 10 +- .../sql-tests/results/pgSQL/float8.sql.out | 5 +- .../sql-tests/results/pgSQL/int4.sql.out | 276 ++++++++---------- .../sql-tests/results/pgSQL/int8.sql.out | 50 ++-- .../apache/spark/sql/SQLQueryTestSuite.scala | 13 + 10 files changed, 193 insertions(+), 209 deletions(-) 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 baa98171e265f..d1943f02f85e5 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 @@ -537,7 +537,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (longValue == longValue.toShort) { longValue.toShort } else { - throw new ArithmeticException(s"Casting $t to short causes overflow.") + throw new ArithmeticException(s"Casting $t to short causes overflow") } }) case TimestampType => @@ -548,12 +548,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) } catch { case _: ArithmeticException => - throw new ArithmeticException(s"Casting $b to short causes overflow.") + throw new ArithmeticException(s"Casting $b to short causes overflow") } if (intValue == intValue.toShort) { intValue.toShort } else { - throw new ArithmeticException(s"Casting $b to short causes overflow.") + throw new ArithmeticException(s"Casting $b to short causes overflow") } case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort @@ -578,7 +578,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (longValue == longValue.toByte) { longValue.toByte } else { - throw new ArithmeticException(s"Casting $t to byte causes overflow.") + throw new ArithmeticException(s"Casting $t to byte causes overflow") } }) case TimestampType => @@ -589,12 +589,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) } catch { case _: ArithmeticException => - throw new ArithmeticException(s"Casting $b to byte causes overflow.") + throw new ArithmeticException(s"Casting $b to byte causes overflow") } if (intValue == intValue.toByte) { intValue.toByte } else { - throw new ArithmeticException(s"Casting $b to byte causes overflow.") + throw new ArithmeticException(s"Casting $b to byte causes overflow") } case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte @@ -1275,7 +1275,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if ($longValue == ($integralType) $longValue) { $evPrim = ($integralType) $longValue; } else { - throw new ArithmeticException("Casting $c to $integralType causes overflow"); + throw new ArithmeticException("Casting " + $c + " to $integralType causes overflow"); } """ } else { @@ -1300,7 +1300,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if ($c == ($integralType) $c) { $evPrim = ($integralType) $c; } else { - throw new ArithmeticException("Casting $c to $integralType causes overflow"); + throw new ArithmeticException("Casting " + $c + " to $integralType causes overflow"); } """ } @@ -1335,7 +1335,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if ($mathClass.floor($c) <= $max && $mathClass.ceil($c) >= $min) { $evPrim = ($integralType) $c; } else { - throw new ArithmeticException("Casting $c to $integralType causes overflow"); + throw new ArithmeticException("Casting " + $c + " to $integralType causes overflow"); } """ } 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 a13929bb772f9..2ed82d99fe2bb 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 @@ -150,9 +150,10 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { def calendarIntervalMethod: String = sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") - /** Name of the function for the exact version of this expression in [[Math]]. */ - def exactMathMethod: String = - sys.error("BinaryArithmetics must override either exactMathMethod 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 + // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. + def exactMathMethod: Option[String] = None override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match { case _: DecimalType => @@ -182,9 +183,9 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { }) case IntegerType | LongType => nullSafeCodeGen(ctx, ev, (eval1, eval2) => { - val operation = if (checkOverflow) { + val operation = if (checkOverflow && exactMathMethod.isDefined) { val mathClass = classOf[Math].getName - s"$mathClass.$exactMathMethod($eval1, $eval2)" + s"$mathClass.${exactMathMethod.get}($eval1, $eval2)" } else { s"$eval1 $symbol $eval2" } @@ -235,7 +236,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic { } } - override def exactMathMethod: String = "addExact" + override def exactMathMethod: Option[String] = Some("addExact") } @ExpressionDescription( @@ -265,7 +266,7 @@ case class Subtract(left: Expression, right: Expression) extends BinaryArithmeti } } - override def exactMathMethod: String = "subtractExact" + override def exactMathMethod: Option[String] = Some("subtractExact") } @ExpressionDescription( @@ -286,7 +287,7 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti protected override def nullSafeEval(input1: Any, input2: Any): Any = numeric.times(input1, input2) - override def exactMathMethod: String = "multiplyExact" + override def exactMathMethod: Option[String] = Some("multiplyExact") } // Common base trait for Divide and Remainder, since these two classes are almost identical diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 6445820f1237b..3136669f64803 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -239,7 +239,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { def toByte: Byte = toLong.toByte private def overflowException(dataType: String) = - throw new ArithmeticException(s"Casting $this to $dataType causes overflow.") + throw new ArithmeticException(s"Casting $this to $dataType causes overflow") /** * @return the Byte value that is equal to the rounded decimal. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala index 9ff55b7f6e6ec..518255ecc42cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala @@ -114,13 +114,13 @@ object LongExactNumeric extends LongIsIntegral with Ordering.LongOrdering { if (x == x.toInt) { x.toInt } else { - throw new ArithmeticException(s"Casting $x to int causes overflow.") + throw new ArithmeticException(s"Casting $x to int causes overflow") } } object FloatExactNumeric extends FloatIsFractional with Ordering.FloatOrdering { private def overflowException(x: Float, dataType: String) = - throw new ArithmeticException(s"Casting $x to $dataType causes overflow.") + throw new ArithmeticException(s"Casting $x to $dataType causes overflow") private val intUpperBound = Int.MaxValue.toFloat private val intLowerBound = Int.MinValue.toFloat @@ -152,7 +152,7 @@ object FloatExactNumeric extends FloatIsFractional with Ordering.FloatOrdering { object DoubleExactNumeric extends DoubleIsFractional with Ordering.DoubleOrdering { private def overflowException(x: Double, dataType: String) = - throw new ArithmeticException(s"Casting $x to $dataType causes overflow.") + throw new ArithmeticException(s"Casting $x to $dataType causes overflow") private val intUpperBound = Int.MaxValue.toDouble private val intLowerBound = Int.MinValue.toDouble diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql index 1012db72e1873..1c2320ff7fad6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql @@ -33,11 +33,6 @@ INSERT INTO INT4_TBL VALUES ('-2147483647'); -- INSERT INTO INT4_TBL(f1) VALUES ('123 5'); -- INSERT INTO INT4_TBL(f1) VALUES (''); --- We cannot test this when failOnOverFlow=true here --- because exception happens in the executors and the --- output stacktrace cannot have an exact match -set spark.sql.arithmeticOperations.failOnOverFlow=false; - SELECT '' AS five, * FROM INT4_TBL; SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0'); diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out index 6e47cff91a7d5..c205f70a65f2d 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out @@ -334,9 +334,10 @@ struct -- !query 37 SELECT int(float('-2147483900')) -- !query 37 schema -struct +struct<> -- !query 37 output --2147483648 +java.lang.ArithmeticException +Casting -2.1474839E9 to int causes overflow -- !query 38 @@ -366,9 +367,10 @@ struct -- !query 41 SELECT bigint(float('-9223380000000000000')) -- !query 41 schema -struct +struct<> -- !query 41 output --9223372036854775808 +java.lang.ArithmeticException +Casting -9.22338E18 to int causes overflow -- !query 42 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out index b4ea3c1ad1cab..9a785670fa950 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out @@ -826,9 +826,10 @@ struct -- !query 93 SELECT bigint(double('-9223372036854780000')) -- !query 93 schema -struct +struct<> -- !query 93 output --9223372036854775808 +java.lang.ArithmeticException +Casting -9.22337203685478E18 to long causes overflow -- !query 94 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out index 879b3c626ec1b..52221d58392f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 54 +-- Number of queries: 53 -- !query 0 @@ -51,27 +51,30 @@ struct<> -- !query 6 -set spark.sql.arithmeticOperations.failOnOverFlow=false +SELECT '' AS five, * FROM INT4_TBL -- !query 6 schema -struct +struct -- !query 6 output -spark.sql.arithmeticOperations.failOnOverFlow false + -123456 + -2147483647 + 0 + 123456 + 2147483647 -- !query 7 -SELECT '' AS five, * FROM INT4_TBL +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') -- !query 7 schema -struct +struct -- !query 7 output -123456 -2147483647 - 0 123456 2147483647 -- !query 8 -SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') -- !query 8 schema struct -- !query 8 output @@ -82,18 +85,15 @@ struct -- !query 9 -SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') -- !query 9 schema -struct +struct -- !query 9 output - -123456 - -2147483647 - 123456 - 2147483647 + 0 -- !query 10 -SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') -- !query 10 schema struct -- !query 10 output @@ -101,15 +101,16 @@ struct -- !query 11 -SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') -- !query 11 schema -struct +struct -- !query 11 output - 0 + -123456 + -2147483647 -- !query 12 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') -- !query 12 schema struct -- !query 12 output @@ -118,16 +119,17 @@ struct -- !query 13 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') -- !query 13 schema -struct +struct -- !query 13 output -123456 -2147483647 + 0 -- !query 14 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') -- !query 14 schema struct -- !query 14 output @@ -137,17 +139,16 @@ struct -- !query 15 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') -- !query 15 schema -struct +struct -- !query 15 output - -123456 - -2147483647 - 0 + 123456 + 2147483647 -- !query 16 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') -- !query 16 schema struct -- !query 16 output @@ -156,16 +157,17 @@ struct -- !query 17 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') -- !query 17 schema -struct +struct -- !query 17 output + 0 123456 2147483647 -- !query 18 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') -- !query 18 schema struct -- !query 18 output @@ -175,81 +177,75 @@ struct -- !query 19 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') -- !query 19 schema -struct +struct -- !query 19 output - 0 - 123456 2147483647 -- !query 20 -SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') -- !query 20 schema -struct +struct -- !query 20 output - 2147483647 + -123456 + 0 + 123456 -- !query 21 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i -- !query 21 schema -struct +struct<> -- !query 21 output - -123456 - 0 - 123456 +java.lang.ArithmeticException +integer overflow -- !query 22 SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 -- !query 22 schema struct -- !query 22 output -123456 -246912 - -2147483647 2 0 0 123456 246912 - 2147483647 -2 -- !query 23 -SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i -WHERE abs(f1) < 1073741824 +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i -- !query 23 schema -struct +struct<> -- !query 23 output - -123456 -246912 - 0 0 - 123456 246912 +java.lang.ArithmeticException +integer overflow -- !query 24 SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 -- !query 24 schema struct -- !query 24 output -123456 -246912 - -2147483647 2 0 0 123456 246912 - 2147483647 -2 -- !query 25 -SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i -WHERE abs(f1) < 1073741824 +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i -- !query 25 schema -struct +struct<> -- !query 25 output - -123456 -246912 - 0 0 - 123456 246912 +java.lang.ArithmeticException +integer overflow -- !query 26 SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 -- !query 26 schema struct -- !query 26 output @@ -257,23 +253,20 @@ struct -2147483647 -2147483645 0 2 123456 123458 - 2147483647 -2147483647 -- !query 27 -SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i -WHERE f1 < 2147483646 +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i -- !query 27 schema -struct +struct<> -- !query 27 output - -123456 -123454 - -2147483647 -2147483645 - 0 2 - 123456 123458 +java.lang.ArithmeticException +integer overflow -- !query 28 SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 -- !query 28 schema struct -- !query 28 output @@ -281,71 +274,64 @@ struct -2147483647 -2147483645 0 2 123456 123458 - 2147483647 -2147483647 -- !query 29 -SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i -WHERE f1 < 2147483646 +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i -- !query 29 schema -struct +struct<> -- !query 29 output - -123456 -123454 - -2147483647 -2147483645 - 0 2 - 123456 123458 +java.lang.ArithmeticException +integer overflow -- !query 30 SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 -- !query 30 schema struct -- !query 30 output -123456 -123458 - -2147483647 2147483647 0 -2 123456 123454 2147483647 2147483645 -- !query 31 -SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i -WHERE f1 > -2147483647 +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i -- !query 31 schema -struct +struct<> -- !query 31 output - -123456 -123458 - 0 -2 - 123456 123454 - 2147483647 2147483645 +java.lang.ArithmeticException +integer overflow -- !query 32 SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 -- !query 32 schema struct -- !query 32 output -123456 -123458 - -2147483647 2147483647 0 -2 123456 123454 2147483647 2147483645 -- !query 33 -SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i -WHERE f1 > -2147483647 +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema struct -- !query 33 output - -123456 -123458 - 0 -2 - 123456 123454 - 2147483647 2147483645 + -123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 -- !query 34 -SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema struct -- !query 34 output @@ -357,51 +343,47 @@ struct -- !query 35 -SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i +SELECT -2+3 AS one -- !query 35 schema -struct +struct -- !query 35 output - -123456 -61728 - -2147483647 -1073741823 - 0 0 - 123456 61728 - 2147483647 1073741823 +1 -- !query 36 -SELECT -2+3 AS one +SELECT 4-2 AS two -- !query 36 schema -struct +struct -- !query 36 output -1 +2 -- !query 37 -SELECT 4-2 AS two +SELECT 2- -1 AS three -- !query 37 schema -struct +struct -- !query 37 output -2 +3 -- !query 38 -SELECT 2- -1 AS three +SELECT 2 - -2 AS four -- !query 38 schema -struct +struct -- !query 38 output -3 +4 -- !query 39 -SELECT 2 - -2 AS four +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true -- !query 39 schema -struct +struct -- !query 39 output -4 +true -- !query 40 -SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true -- !query 40 schema struct -- !query 40 output @@ -409,7 +391,7 @@ true -- !query 41 -SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true -- !query 41 schema struct -- !query 41 output @@ -417,78 +399,70 @@ true -- !query 42 -SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true +SELECT int('1000') < int('999') AS `false` -- !query 42 schema -struct +struct -- !query 42 output -true +false -- !query 43 -SELECT int('1000') < int('999') AS `false` +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten -- !query 43 schema -struct +struct -- !query 43 output -false +10 -- !query 44 -SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten +SELECT 2 + 2 / 2 AS three -- !query 44 schema -struct +struct -- !query 44 output -10 +3 -- !query 45 -SELECT 2 + 2 / 2 AS three +SELECT (2 + 2) / 2 AS two -- !query 45 schema -struct +struct -- !query 45 output -3 +2 -- !query 46 -SELECT (2 + 2) / 2 AS two +SELECT string(shiftleft(int(-1), 31)) -- !query 46 schema -struct +struct -- !query 46 output -2 +-2147483648 -- !query 47 -SELECT string(shiftleft(int(-1), 31)) +SELECT string(int(shiftleft(int(-1), 31))+1) -- !query 47 schema -struct +struct -- !query 47 output --2147483648 +-2147483647 -- !query 48 -SELECT string(int(shiftleft(int(-1), 31))+1) +SELECT int(-2147483648) % int(-1) -- !query 48 schema -struct +struct<(CAST(-2147483648 AS INT) % CAST(-1 AS INT)):int> -- !query 48 output --2147483647 +0 -- !query 49 -SELECT int(-2147483648) % int(-1) +SELECT int(-2147483648) % smallint(-1) -- !query 49 schema -struct<(CAST(-2147483648 AS INT) % CAST(-1 AS INT)):int> +struct<(CAST(-2147483648 AS INT) % CAST(CAST(-1 AS SMALLINT) AS INT)):int> -- !query 49 output 0 -- !query 50 -SELECT int(-2147483648) % smallint(-1) --- !query 50 schema -struct<(CAST(-2147483648 AS INT) % CAST(CAST(-1 AS SMALLINT) AS INT)):int> --- !query 50 output -0 - - --- !query 51 SELECT x, int(x) AS int4_value FROM (VALUES double(-2.5), double(-1.5), @@ -497,9 +471,9 @@ FROM (VALUES double(-2.5), double(0.5), double(1.5), double(2.5)) t(x) --- !query 51 schema +-- !query 50 schema struct --- !query 51 output +-- !query 50 output -0.5 0 -1.5 -1 -2.5 -2 @@ -509,7 +483,7 @@ struct 2.5 2 --- !query 52 +-- !query 51 SELECT x, int(x) AS int4_value FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), @@ -518,9 +492,9 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(0.5 as decimal(38, 18)), cast(1.5 as decimal(38, 18)), cast(2.5 as decimal(38, 18))) t(x) --- !query 52 schema +-- !query 51 schema struct --- !query 52 output +-- !query 51 output -0.5 0 -1.5 -1 -2.5 -2 @@ -530,9 +504,9 @@ struct 2.5 2 --- !query 53 +-- !query 52 DROP TABLE INT4_TBL --- !query 53 schema +-- !query 52 schema struct<> --- !query 53 output +-- !query 52 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out index fc9f1474eb26c..20bb5dbda645f 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out @@ -389,13 +389,10 @@ struct -- !query 40 SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL -- !query 40 schema -struct +struct<> -- !query 40 output - 123 456 56088 - 123 4567890123456789 561850485185185047 - 4567890123456789 -4567890123456789 -4868582358072306617 - 4567890123456789 123 561850485185185047 - 4567890123456789 4567890123456789 4868582358072306617 +java.lang.ArithmeticException +long overflow -- !query 41 @@ -604,12 +601,10 @@ struct -- !query 60 SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 -- !query 60 schema -struct +struct<> -- !query 60 output --869367531 --869367531 --869367531 -123 +java.lang.ArithmeticException +Casting 4567890123456789 to int causes overflow -- !query 61 @@ -623,12 +618,10 @@ struct -- !query 62 SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 -- !query 62 schema -struct +struct<> -- !query 62 output --32491 --32491 --32491 -123 +java.lang.ArithmeticException +Casting 4567890123456789 to short causes overflow -- !query 63 @@ -662,9 +655,10 @@ struct -- !query 66 SELECT CAST(double('922337203685477580700.0') AS bigint) -- !query 66 schema -struct +struct<> -- !query 66 output -9223372036854775807 +java.lang.ArithmeticException +Casting 9.223372036854776E20 to long causes overflow -- !query 67 @@ -728,17 +722,19 @@ struct -- !query 72 SELECT string(int(shiftleft(bigint(-1), 63))+1) -- !query 72 schema -struct +struct<> -- !query 72 output -1 +java.lang.ArithmeticException +Casting -9223372036854775808 to int causes overflow -- !query 73 SELECT bigint((-9223372036854775808)) * bigint((-1)) -- !query 73 schema -struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(-1 AS BIGINT)):bigint> +struct<> -- !query 73 output --9223372036854775808 +java.lang.ArithmeticException +long overflow -- !query 74 @@ -760,9 +756,10 @@ struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(-1 AS BIGINT)):bigint> -- !query 76 SELECT bigint((-9223372036854775808)) * int((-1)) -- !query 76 schema -struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +struct<> -- !query 76 output --9223372036854775808 +java.lang.ArithmeticException +long overflow -- !query 77 @@ -784,9 +781,10 @@ struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS INT) AS BIGINT)): -- !query 79 SELECT bigint((-9223372036854775808)) * smallint((-1)) -- !query 79 schema -struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +struct<> -- !query 79 output --9223372036854775808 +java.lang.ArithmeticException +long overflow -- !query 80 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 28ca0edaef871..cc92d6556387d 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,12 +22,14 @@ import java.util.{Locale, TimeZone} import scala.util.control.NonFatal +import org.apache.spark.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 +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -308,6 +310,11 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) + localSparkSession.conf.set(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, true) + // Propagate the SQL conf FAIL_ON_INTEGRAL_TYPE_OVERFLOW to executor. + // TODO: remove this after SPARK-29122 is resolved. + localSparkSession.sparkContext.setLocalProperty( + SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, "true") case _ => } @@ -413,6 +420,12 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // 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"))) + 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)) 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)) From b48ef7a9fb1dec720d0f94ce7066fda106ab7bfe Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Wed, 18 Sep 2019 08:44:44 -0500 Subject: [PATCH 32/87] [SPARK-28799][DOC] Documentation for Truncate command ### What changes were proposed in this pull request? Document TRUNCATE statement in SQL Reference Guide. ### Why are the changes needed? Adding documentation for SQL reference. ### Does this PR introduce any user-facing change? yes Before: There was no documentation for this. After. ![image (4)](https://user-images.githubusercontent.com/51401130/64956929-5e057780-d8a9-11e9-89a3-2d02c942b9ad.png) ![image (5)](https://user-images.githubusercontent.com/51401130/64956942-61006800-d8a9-11e9-9767-6164eabfdc2c.png) ### How was this patch tested? Used jekyll build and serve to verify. Closes #25557 from PavithraRamachandran/truncate_doc. Lead-authored-by: Pavithra Ramachandran Co-authored-by: pavithra Signed-off-by: Sean Owen --- docs/sql-ref-syntax-ddl-truncate-table.md | 66 ++++++++++++++++++++++- 1 file changed, 65 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 2704259391e94..4b4094ab708e5 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -19,4 +19,68 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The `TRUNCATE TABLE` statement removes all the rows from a table or partition(s). The table must not be a view +or an external/temporary table. In order to truncate multiple partitions at once, the user can specify the partitions +in `partition_spec`. If no `partition_spec` is specified it will remove all partitions in the table. + +### Syntax +{% highlight sql %} +TRUNCATE TABLE table_name [PARTITION partition_spec]; +{% endhighlight %} + +### Parameters +
    +
    table_name
    +
    The name of an existing table.
    +
    + +
    +
    PARTITION ( partition_spec :[ partition_column = partition_col_value, partition_column = partition_col_value, ...] )
    +
    Specifies one or more partition column and value pairs. The partition value is optional.
    +
    + + +### Examples +{% highlight sql %} + +--Create table Student with partition +CREATE TABLE Student ( name String, rollno INT) PARTITIONED BY (age int); + +SELECT * from Student; ++-------+---------+------+--+ +| name | rollno | age | ++-------+---------+------+--+ +| ABC | 1 | 10 | +| DEF | 2 | 10 | +| XYZ | 3 | 12 | ++-------+---------+------+--+ + +-- Removes all rows from the table in the partion specified +TRUNCATE TABLE Student partition(age=10); + +--After truncate execution, records belonging to partition age=10 are removed +SELECT * from Student; ++-------+---------+------+--+ +| name | rollno | age | ++-------+---------+------+--+ +| XYZ | 3 | 12 | ++-------+---------+------+--+ + +-- Removes all rows from the table from all partitions +TRUNCATE TABLE Student; + +SELECT * from Student; ++-------+---------+------+--+ +| name | rollno | age | ++-------+---------+------+--+ ++-------+---------+------+--+ +No rows selected + +{% endhighlight %} + + +### Related Statements +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) +- [ALTER TABLE](sql-ref-syntax-ddl-alter-tabley.html) + From 600a2a4ae585a47c822efe42f4fcccb048d37864 Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Wed, 18 Sep 2019 09:11:15 -0500 Subject: [PATCH 33/87] [SPARK-28972][DOCS] Updating unit description in configurations, to maintain consistency ### What changes were proposed in this pull request? Updating unit description in configurations, inorder to maintain consistency across configurations. ### Why are the changes needed? the description does not mention about suffix that can be mentioned while configuring this value. For better user understanding ### Does this PR introduce any user-facing change? yes. Doc description ### How was this patch tested? generated document and checked. ![Screenshot from 2019-09-05 11-09-17](https://user-images.githubusercontent.com/51401130/64314853-07a55880-cfce-11e9-8af0-6416a50b0188.png) Closes #25689 from PavithraRamachandran/heapsize_config. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../apache/spark/internal/config/package.scala | 3 ++- docs/configuration.md | 18 ++++++++++-------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 158a4b7cfa55a..6011901d1d53d 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -243,7 +243,8 @@ package object config { .createWithDefault(false) private[spark] val MEMORY_OFFHEAP_SIZE = ConfigBuilder("spark.memory.offHeap.size") - .doc("The absolute amount of memory in bytes which can be used for off-heap allocation. " + + .doc("The absolute amount of memory which can be used for off-heap allocation, " + + " in bytes unless otherwise specified. " + "This setting has no impact on heap memory usage, so if your executors' total memory " + "consumption must fit within some hard limit then be sure to shrink your JVM heap size " + "accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true.") diff --git a/docs/configuration.md b/docs/configuration.md index 5cf42d5fe8dd2..e0b7386ac9bcc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -866,7 +866,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.service.index.cache.size 100m - Cache entries limited to the specified memory footprint in bytes. + Cache entries limited to the specified memory footprint, in bytes unless otherwise specified. @@ -1207,16 +1207,18 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.lz4.blockSize 32k - Block size in bytes used in LZ4 compression, in the case when LZ4 compression codec + Block size used in LZ4 compression, in the case when LZ4 compression codec is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used. + Default unit is bytes, unless otherwise specified. spark.io.compression.snappy.blockSize 32k - Block size in bytes used in Snappy compression, in the case when Snappy compression codec - is used. Lowering this block size will also lower shuffle memory usage when Snappy is used. + Block size in Snappy compression, in the case when Snappy compression codec is used. + Lowering this block size will also lower shuffle memory usage when Snappy is used. + Default unit is bytes, unless otherwise specified. @@ -1384,7 +1386,7 @@ Apart from these, the following properties are also available, and may be useful spark.memory.offHeap.size 0 - The absolute amount of memory in bytes which can be used for off-heap allocation. + The absolute amount of memory which can be used for off-heap allocation, in bytes unless otherwise specified. This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true. @@ -1568,9 +1570,9 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryMapThreshold 2m - Size in bytes of a block above which Spark memory maps when reading a block from disk. - This prevents Spark from memory mapping very small blocks. In general, memory - mapping has high overhead for blocks close to or below the page size of the operating system. + Size of a block above which Spark memory maps when reading a block from disk. Default unit is bytes, + unless specified otherwise. This prevents Spark from memory mapping very small blocks. In general, + memory mapping has high overhead for blocks close to or below the page size of the operating system. From 12e1583093bbd8f6b7146969f93bbff09d583561 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 18 Sep 2019 09:22:13 -0500 Subject: [PATCH 34/87] [SPARK-28927][ML] Rethrow block mismatch exception in ALS when input data is nondeterministic ### What changes were proposed in this pull request? Fitting ALS model can be failed due to nondeterministic input data. Currently the failure is thrown by an ArrayIndexOutOfBoundsException which is not explainable for end users what is wrong in fitting. This patch catches this exception and rethrows a more explainable one, when the input data is nondeterministic. Because we may not exactly know the output deterministic level of RDDs produced by user code, this patch also adds a note to Scala/Python/R ALS document about the training data deterministic level. ### Why are the changes needed? ArrayIndexOutOfBoundsException was observed during fitting ALS model. It was caused by mismatching between in/out user/item blocks during computing ratings. If the training RDD output is nondeterministic, when fetch failure is happened, rerun part of training RDD can produce inconsistent user/item blocks. This patch is needed to notify users ALS fitting on nondeterministic input. ### Does this PR introduce any user-facing change? Yes. When fitting ALS model on nondeterministic input data, previously if rerun happens, users would see ArrayIndexOutOfBoundsException caused by mismatch between In/Out user/item blocks. After this patch, a SparkException with more clear message will be thrown, and original ArrayIndexOutOfBoundsException is wrapped. ### How was this patch tested? Tested on development cluster. Closes #25789 from viirya/als-indeterminate-input. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Sean Owen --- R/pkg/R/mllib_recommendation.R | 6 ++++ .../apache/spark/ml/recommendation/ALS.scala | 32 +++++++++++++++++-- .../spark/mllib/recommendation/ALS.scala | 7 ++++ python/pyspark/ml/recommendation.py | 7 ++++ 4 files changed, 49 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/mllib_recommendation.R b/R/pkg/R/mllib_recommendation.R index 9a77b07462585..d238ff93ed245 100644 --- a/R/pkg/R/mllib_recommendation.R +++ b/R/pkg/R/mllib_recommendation.R @@ -82,6 +82,12 @@ setClass("ALSModel", representation(jobj = "jobj")) #' statsS <- summary(modelS) #' } #' @note spark.als since 2.1.0 +#' @note the input rating dataframe to the ALS implementation should be deterministic. +#' Nondeterministic data can cause failure during fitting ALS model. For example, +#' an order-sensitive operation like sampling after a repartition makes dataframe output +#' nondeterministic, like \code{sample(repartition(df, 2L), FALSE, 0.5, 1618L)}. +#' Checkpointing sampled dataframe or adding a sort before sampling can help make the +#' dataframe deterministic. setMethod("spark.als", signature(data = "SparkDataFrame"), function(data, ratingCol = "rating", userCol = "user", itemCol = "item", rank = 10, regParam = 0.1, maxIter = 10, nonnegative = FALSE, 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 fb7334d41ba44..33097f322444b 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 @@ -31,7 +31,7 @@ import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.json4s.JsonDSL._ -import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} @@ -42,7 +42,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.mllib.optimization.NNLS -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -564,6 +564,13 @@ object ALSModel extends MLReadable[ALSModel] { * r is greater than 0 and 0 if r is less than or equal to 0. The ratings then act as 'confidence' * values related to strength of indicated user * preferences rather than explicit ratings given to items. + * + * Note: the input rating dataset to the ALS implementation should be deterministic. + * Nondeterministic data can cause failure during fitting ALS model. + * For example, an order-sensitive operation like sampling after a repartition makes dataset + * output nondeterministic, like `dataset.repartition(2).sample(false, 0.5, 1618)`. + * Checkpointing sampled dataset or adding a sort before sampling can help make the dataset + * deterministic. */ @Since("1.3.0") class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams @@ -1666,6 +1673,13 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { } } val merged = srcOut.groupByKey(new ALSPartitioner(dstInBlocks.partitions.length)) + + // SPARK-28927: Nondeterministic RDDs causes inconsistent in/out blocks in case of rerun. + // It can cause runtime error when matching in/out user/item blocks. + val isBlockRDDNondeterministic = + dstInBlocks.outputDeterministicLevel == DeterministicLevel.INDETERMINATE || + srcOutBlocks.outputDeterministicLevel == DeterministicLevel.INDETERMINATE + dstInBlocks.join(merged).mapValues { case (InBlock(dstIds, srcPtrs, srcEncodedIndices, ratings), srcFactors) => val sortedSrcFactors = new Array[FactorBlock](numSrcBlocks) @@ -1686,7 +1700,19 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { val encoded = srcEncodedIndices(i) val blockId = srcEncoder.blockId(encoded) val localIndex = srcEncoder.localIndex(encoded) - val srcFactor = sortedSrcFactors(blockId)(localIndex) + var srcFactor: Array[Float] = null + try { + srcFactor = sortedSrcFactors(blockId)(localIndex) + } catch { + case a: ArrayIndexOutOfBoundsException if isBlockRDDNondeterministic => + val errMsg = "A failure detected when matching In/Out blocks of users/items. " + + "Because at least one In/Out block RDD is found to be nondeterministic now, " + + "the issue is probably caused by nondeterministic input data. You can try to " + + "checkpoint training data to make it deterministic. If you do `repartition` + " + + "`sample` or `randomSplit`, you can also try to sort it before `sample` or " + + "`randomSplit` to make it deterministic." + throw new SparkException(errMsg, a) + } val rating = ratings(i) if (implicitPrefs) { // Extension to the original paper to handle rating < 0. confidence is a function diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 12870f819b147..f3f15ba0d0f2c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -62,6 +62,13 @@ case class Rating @Since("0.8.0") ( * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of * indicated user * preferences rather than explicit ratings given to items. + * + * Note: the input rating RDD to the ALS implementation should be deterministic. + * Nondeterministic data can cause failure during fitting ALS model. + * For example, an order-sensitive operation like sampling after a repartition makes RDD + * output nondeterministic, like `rdd.repartition(2).sample(false, 0.5, 1618)`. + * Checkpointing sampled RDD or adding a sort before sampling can help make the RDD + * deterministic. */ @Since("0.8.0") class ALS private ( diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index bf2716485df9c..676662da8c316 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -67,6 +67,13 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha indicated user preferences rather than explicit ratings given to items. + .. note:: the input rating dataframe to the ALS implementation should be deterministic. + Nondeterministic data can cause failure during fitting ALS model. + For example, an order-sensitive operation like sampling after a repartition makes + dataframe output nondeterministic, like `df.repartition(2).sample(False, 0.5, 1618)`. + Checkpointing sampled dataframe or adding a sort before sampling can help make the + dataframe deterministic. + >>> df = spark.createDataFrame( ... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)], ... ["user", "item", "rating"]) From c2734ab1fcc523507174c4f9e7368c1e7a2368e8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Sep 2019 23:30:59 +0900 Subject: [PATCH 35/87] [SPARK-29012][SQL] Support special timestamp values ### What changes were proposed in this pull request? Supported special string values for `TIMESTAMP` type. They are simply notational shorthands that will be converted to ordinary timestamp values when read. The following string values are supported: - `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: ```sql spark-sql> SELECT timestamp 'tomorrow'; 2019-09-07 00:00:00 ``` ### Why are the changes needed? To maintain feature parity with PostgreSQL, see [8.5.1.4. Special Values](https://www.postgresql.org/docs/12/datatype-datetime.html) ### Does this PR introduce any user-facing change? Previously, the parser fails on the special values with the error: ```sql spark-sql> select timestamp 'today'; Error in query: Cannot parse the TIMESTAMP value: today(line 1, pos 7) ``` After the changes, the special values are converted to appropriate dates: ```sql spark-sql> select timestamp 'today'; 2019-09-06 00:00:00 ``` ### How was this patch tested? - Added tests to `TimestampFormatterSuite` to check parsing special values from regular strings. - Tests in `DateTimeUtilsSuite` check parsing those values from `UTF8String` - Uncommented tests in `timestamp.sql` Closes #25716 from MaxGekk/timestamp-special-values. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../expressions/datetimeExpressions.scala | 4 +- .../sql/catalyst/util/DateTimeUtils.scala | 67 ++++++ .../catalyst/util/TimestampFormatter.scala | 21 +- .../catalyst/util/DateTimeUtilsSuite.scala | 36 +++- .../sql/util/TimestampFormatterSuite.scala | 28 ++- .../sql-tests/inputs/pgSQL/timestamp.sql | 29 +-- .../sql-tests/results/pgSQL/timestamp.sql.out | 190 ++++++++++++++---- .../apache/spark/sql/CsvFunctionsSuite.scala | 10 + .../apache/spark/sql/JsonFunctionsSuite.scala | 10 + 9 files changed, 317 insertions(+), 78 deletions(-) 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 597be5dc97b43..89a6d23b1d73d 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 @@ -98,9 +98,7 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { override def dataType: DataType = TimestampType - override def eval(input: InternalRow): Any = { - instantToMicros(Instant.now()) - } + override def eval(input: InternalRow): Any = currentTimestamp() override def prettyName: String = "current_timestamp" } 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 65a9bee5eaedd..a82471aae652d 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 @@ -17,12 +17,14 @@ package org.apache.spark.sql.catalyst.util +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.time._ import java.time.temporal.{ChronoField, ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ +import scala.util.Try import scala.util.control.NonFatal import org.apache.spark.sql.types.Decimal @@ -218,6 +220,8 @@ object DateTimeUtils { var i = 0 var currentSegmentValue = 0 val bytes = s.trim.getBytes + val specialTimestamp = convertSpecialTimestamp(bytes, timeZoneId) + if (specialTimestamp.isDefined) return specialTimestamp var j = 0 var digitsMilli = 0 var justTime = false @@ -848,4 +852,67 @@ object DateTimeUtils { val sinceEpoch = BigDecimal(timestamp) / MICROS_PER_SECOND + offset new Decimal().set(sinceEpoch, 20, 6) } + + def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) + + private def today(zoneId: ZoneId): ZonedDateTime = { + Instant.now().atZone(zoneId).`with`(LocalTime.MIDNIGHT) + } + + private val specialValueRe = """(\p{Alpha}+)\p{Blank}*(.*)""".r + + /** + * Extracts special values from an input string ignoring case. + * @param input - a trimmed string + * @param zoneId - zone identifier used to get the current date. + * @return some special value in lower case or None. + */ + private def extractSpecialValue(input: String, zoneId: ZoneId): Option[String] = { + def isValid(value: String, timeZoneId: String): Boolean = { + // Special value can be without any time zone + if (timeZoneId.isEmpty) return true + // "now" must not have the time zone field + if (value.compareToIgnoreCase("now") == 0) return false + // If the time zone field presents in the input, it must be resolvable + try { + getZoneId(timeZoneId) + true + } catch { + case NonFatal(_) => false + } + } + + assert(input.trim.length == input.length) + if (input.length < 3 || !input(0).isLetter) return None + input match { + case specialValueRe(v, z) if isValid(v, z) => Some(v.toLowerCase(Locale.US)) + case _ => None + } + } + + /** + * Converts notational shorthands that are converted to ordinary timestamps. + * @param input - a trimmed string + * @param zoneId - zone identifier used to get the current date. + * @return some of microseconds since the epoch if the conversion completed + * successfully otherwise None. + */ + def convertSpecialTimestamp(input: String, zoneId: ZoneId): Option[SQLTimestamp] = { + extractSpecialValue(input, zoneId).flatMap { + case "epoch" => Some(0) + case "now" => Some(currentTimestamp()) + case "today" => Some(instantToMicros(today(zoneId).toInstant)) + case "tomorrow" => Some(instantToMicros(today(zoneId).plusDays(1).toInstant)) + case "yesterday" => Some(instantToMicros(today(zoneId).minusDays(1).toInstant)) + case _ => None + } + } + + private def convertSpecialTimestamp(bytes: Array[Byte], zoneId: ZoneId): Option[SQLTimestamp] = { + if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) { + convertSpecialTimestamp(new String(bytes, StandardCharsets.UTF_8), zoneId) + } else { + None + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 3dfcf1a33eb56..5be4807083fa3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -22,9 +22,11 @@ import java.time._ import java.time.format.DateTimeParseException import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries -import java.util.{Locale, TimeZone} +import java.util.Locale import java.util.concurrent.TimeUnit.SECONDS +import DateTimeUtils.convertSpecialTimestamp + sealed trait TimestampFormatter extends Serializable { /** * Parses a timestamp in a string and converts it to microseconds. @@ -50,14 +52,17 @@ class Iso8601TimestampFormatter( protected lazy val formatter = getOrCreateFormatter(pattern, locale) override def parse(s: String): Long = { - val parsed = formatter.parse(s) - val parsedZoneId = parsed.query(TemporalQueries.zone()) - val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId - val zonedDateTime = toZonedDateTime(parsed, timeZoneId) - val epochSeconds = zonedDateTime.toEpochSecond - val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) + val specialDate = convertSpecialTimestamp(s.trim, zoneId) + specialDate.getOrElse { + val parsed = formatter.parse(s) + val parsedZoneId = parsed.query(TemporalQueries.zone()) + val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId + val zonedDateTime = toZonedDateTime(parsed, timeZoneId) + val epochSeconds = zonedDateTime.toEpochSecond + val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) - Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + } } override def format(us: Long): String = { 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 056337205ae7e..31fefd613f9c8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -19,16 +19,18 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.ZoneId +import java.time.{LocalDateTime, LocalTime, ZoneId} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit +import org.scalatest.Matchers + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.unsafe.types.UTF8String -class DateTimeUtilsSuite extends SparkFunSuite { +class DateTimeUtilsSuite extends SparkFunSuite with Matchers { val TimeZonePST = TimeZone.getTimeZone("PST") private def defaultZoneId = ZoneId.systemDefault() @@ -142,10 +144,14 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(stringToDate(UTF8String.fromString("1999 08")).isEmpty) } + private def toTimestamp(str: String, zoneId: ZoneId): Option[SQLTimestamp] = { + stringToTimestamp(UTF8String.fromString(str), zoneId) + } + test("string to timestamp") { for (tz <- ALL_TIMEZONES) { def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { - assert(stringToTimestamp(UTF8String.fromString(str), tz.toZoneId) === expected) + assert(toTimestamp(str, tz.toZoneId) === expected) } checkStringToTimestamp("1969-12-31 16:00:00", Option(date(1969, 12, 31, 16, tz = tz))) @@ -271,8 +277,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { UTF8String.fromString("2015-02-29 00:00:00"), defaultZoneId).isEmpty) assert(stringToTimestamp( UTF8String.fromString("2015-04-31 00:00:00"), defaultZoneId).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-02-29"), defaultZoneId).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-04-31"), defaultZoneId).isEmpty) + assert(toTimestamp("2015-02-29", defaultZoneId).isEmpty) + assert(toTimestamp("2015-04-31", defaultZoneId).isEmpty) } test("hours") { @@ -456,8 +462,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { timezone: TimeZone = DateTimeUtils.defaultTimeZone()): Unit = { val truncated = DateTimeUtils.truncTimestamp(inputTS, level, timezone) - val expectedTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected), defaultZoneId) + val expectedTS = toTimestamp(expected, defaultZoneId) assert(truncated === expectedTS.get) } @@ -564,4 +569,21 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(DateTimeUtils.toMillis(-9223372036844776001L) === -9223372036844777L) assert(DateTimeUtils.toMillis(-157700927876544L) === -157700927877L) } + + test("special timestamp values") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + val tolerance = TimeUnit.SECONDS.toMicros(30) + + assert(toTimestamp("Epoch", zoneId).get === 0) + val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant) + toTimestamp("NOW", zoneId).get should be (now +- tolerance) + assert(toTimestamp("now UTC", zoneId) === None) + val today = instantToMicros(LocalDateTime.now(zoneId) + .`with`(LocalTime.MIDNIGHT) + .atZone(zoneId).toInstant) + toTimestamp(" Yesterday", zoneId).get should be (today - MICROS_PER_DAY +- tolerance) + toTimestamp("Today ", zoneId).get should be (today +- tolerance) + toTimestamp(" tomorrow CET ", zoneId).get should be (today + MICROS_PER_DAY +- tolerance) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index c223639a47294..170daa6277c49 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -17,14 +17,18 @@ package org.apache.spark.sql.util -import java.time.{LocalDateTime, ZoneId, ZoneOffset} +import java.time.{LocalDateTime, LocalTime, ZoneOffset} import java.util.concurrent.TimeUnit +import org.scalatest.Matchers + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, instantToMicros, MICROS_PER_DAY} +import org.apache.spark.sql.internal.SQLConf -class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { +class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers { test("parsing timestamps using time zones") { val localDate = "2018-12-02T10:11:12.001234" @@ -131,4 +135,24 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { val micros = DateTimeUtils.instantToMicros(instant) assert(TimestampFormatter(ZoneOffset.UTC).format(micros) === "-0099-01-01 00:00:00") } + + test("special timestamp values") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val zoneId = getZoneId(timeZone) + val formatter = TimestampFormatter(zoneId) + val tolerance = TimeUnit.SECONDS.toMicros(30) + + assert(formatter.parse("EPOCH") === 0) + val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant) + formatter.parse("now") should be (now +- tolerance) + val today = instantToMicros(LocalDateTime.now(zoneId) + .`with`(LocalTime.MIDNIGHT) + .atZone(zoneId).toInstant) + formatter.parse("yesterday CET") should be (today - MICROS_PER_DAY +- tolerance) + formatter.parse(" TODAY ") should be (today +- tolerance) + formatter.parse("Tomorrow ") should be (today + MICROS_PER_DAY +- tolerance) + } + } + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql index 65e8d3280e07c..92bbe14dc1e5f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql @@ -7,7 +7,6 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; --- [SPARK-28141] Timestamp type can not accept special values -- Test shorthand input values -- We can't just "select" the results since they aren't constants; test for -- equality instead. We can do that by running the test inside a transaction @@ -17,22 +16,24 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; -- block is entered exactly at local midnight; then 'now' and 'today' have -- the same values and the counts will come out different. --- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +INSERT INTO TIMESTAMP_TBL VALUES ('now'); -- SELECT pg_sleep(0.1); -- BEGIN; --- INSERT INTO TIMESTAMP_TBL VALUES ('now'); --- INSERT INTO TIMESTAMP_TBL VALUES ('today'); --- INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); --- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +INSERT INTO TIMESTAMP_TBL VALUES ('now'); +INSERT INTO TIMESTAMP_TBL VALUES ('today'); +INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); -- time zone should be ignored by this data type --- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); --- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow zulu'); - --- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; --- SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; --- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +-- [SPARK-29024] Ignore case while resolving time zones +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu'); + +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +-- [SPARK-29025] Support seconds precision by the timestamp type -- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; -- COMMIT; @@ -48,12 +49,12 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; -- SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; -- COMMIT; --- TRUNCATE TIMESTAMP_TBL; +TRUNCATE TABLE TIMESTAMP_TBL; -- Special values -- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); -- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); --- INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); -- [SPARK-27923] Spark SQL insert there obsolete special values to NULL -- Obsolete special values -- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out index 75d9ee8d9c797..db17ee46c6829 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 18 +-- Number of queries: 30 -- !query 0 @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +INSERT INTO TIMESTAMP_TBL VALUES ('now') -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +INSERT INTO TIMESTAMP_TBL VALUES ('now') -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +INSERT INTO TIMESTAMP_TBL VALUES ('today') -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +INSERT INTO TIMESTAMP_TBL VALUES ('yesterday') -- !query 4 schema struct<> -- !query 4 output @@ -43,139 +43,241 @@ struct<> -- !query 5 -SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow') -- !query 5 schema -struct<64:string,d1:timestamp> +struct<> -- !query 5 output + + + +-- !query 6 +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today' +-- !query 8 schema +struct +-- !query 8 output +1 + + +-- !query 9 +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow' +-- !query 9 schema +struct +-- !query 9 output +3 + + +-- !query 10 +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday' +-- !query 10 schema +struct +-- !query 10 output +1 + + +-- !query 11 +TRUNCATE TABLE TIMESTAMP_TBL +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO TIMESTAMP_TBL VALUES ('epoch') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query 17 schema +struct<64:string,d1:timestamp> +-- !query 17 output + 1969-12-31 16:00:00 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 6 +-- !query 18 SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL WHERE d1 > timestamp '1997-01-02' --- !query 6 schema +-- !query 18 schema struct<48:string,d1:timestamp> --- !query 6 output +-- !query 18 output 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 7 +-- !query 19 SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL WHERE d1 < timestamp '1997-01-02' --- !query 7 schema +-- !query 19 schema struct<15:string,d1:timestamp> --- !query 7 output +-- !query 19 output + 1969-12-31 16:00:00 - --- !query 8 +-- !query 20 SELECT '' AS one, d1 FROM TIMESTAMP_TBL WHERE d1 = timestamp '1997-01-02' --- !query 8 schema +-- !query 20 schema struct --- !query 8 output +-- !query 20 output 1997-01-02 00:00:00 --- !query 9 +-- !query 21 SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL WHERE d1 != timestamp '1997-01-02' --- !query 9 schema +-- !query 21 schema struct<63:string,d1:timestamp> --- !query 9 output +-- !query 21 output + 1969-12-31 16:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 10 +-- !query 22 SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL WHERE d1 <= timestamp '1997-01-02' --- !query 10 schema +-- !query 22 schema struct<16:string,d1:timestamp> --- !query 10 output +-- !query 22 output + 1969-12-31 16:00:00 1997-01-02 00:00:00 --- !query 11 +-- !query 23 SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL WHERE d1 >= timestamp '1997-01-02' --- !query 11 schema +-- !query 23 schema struct<49:string,d1:timestamp> --- !query 11 output +-- !query 23 output 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 12 +-- !query 24 SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc --- !query 12 schema +-- !query 24 schema struct --- !query 12 output +-- !query 24 output 2004-02-23 00:00:00 --- !query 13 +-- !query 25 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 13 schema +-- !query 25 schema struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:int> --- !query 13 output +-- !query 25 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 1997-02-10 17:32:01 1997 2 10 17 32 1 2001-09-22 18:19:20 2001 9 22 18 19 20 --- !query 14 +-- !query 26 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 14 schema +-- !query 26 schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> --- !query 14 output +-- !query 26 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 1997-02-10 17:32:01 1 1000 1000000 2001-09-22 18:19:20 3 20000 20000000 --- !query 15 +-- !query 27 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 15 schema +-- !query 27 schema struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> --- !query 15 output +-- !query 27 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 1997-02-10 17:32:01 1997 7 1 2001-09-22 18:19:20 2001 38 6 --- !query 16 +-- !query 28 SELECT make_timestamp(2014,12,28,6,30,45.887) --- !query 16 schema +-- !query 28 schema struct --- !query 16 output +-- !query 28 output 2014-12-28 06:30:45.887 --- !query 17 +-- !query 29 DROP TABLE TIMESTAMP_TBL --- !query 17 schema +-- !query 29 schema struct<> --- !query 17 output +-- !query 29 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 52cf91cfade51..1094d7d23e5ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import java.sql.Timestamp import java.text.SimpleDateFormat import java.util.Locale @@ -181,4 +182,13 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0")))) } } + + test("special timestamp values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(specialValue).toDS() + val readback = input.select(from_csv($"value", lit("t timestamp"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 92a4acc130be5..c61c8109ee8e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import java.sql.Timestamp import java.text.SimpleDateFormat import java.util.Locale @@ -608,4 +609,13 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0")))) } } + + test("special timestamp values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(s"""{"t": "$specialValue"}""").toDS() + val readback = input.select(from_json($"value", lit("t timestamp"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) + } + } } From 203bf9e5698df83a2c7b77ed8bad824784818064 Mon Sep 17 00:00:00 2001 From: Xianjin YE Date: Wed, 18 Sep 2019 23:32:10 +0900 Subject: [PATCH 36/87] [SPARK-19926][PYSPARK] make captured exception from JVM side user friendly MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? The str of `CapaturedException` is now returned by str(self.desc) rather than repr(self.desc), which is more user-friendly. It also handles unicode under python2 specially. ### Why are the changes needed? This is an improvement, and makes exception more human readable in python side. ### Does this PR introduce any user-facing change? Before this pr, select `中文字段` throws exception something likes below: ``` Traceback (most recent call last): File "/Users/advancedxy/code_workspace/github/spark/python/pyspark/sql/tests/test_utils.py", line 34, in test_capture_user_friendly_exception raise e AnalysisException: u"cannot resolve '`\u4e2d\u6587\u5b57\u6bb5`' given input columns: []; line 1 pos 7;\n'Project ['\u4e2d\u6587\u5b57\u6bb5]\n+- OneRowRelation\n" ``` after this pr: ``` Traceback (most recent call last): File "/Users/advancedxy/code_workspace/github/spark/python/pyspark/sql/tests/test_utils.py", line 34, in test_capture_user_friendly_exception raise e AnalysisException: cannot resolve '`中文字段`' given input columns: []; line 1 pos 7; 'Project ['中文字段] +- OneRowRelation ``` ### How was this patch Add a new test to verify unicode are correctly converted and manual checks for thrown exceptions. This pr's credits should go to uncleGen and is based on https://github.com/apache/spark/pull/17267 Closes #25814 from advancedxy/python_exception_19926_and_21045. Authored-by: Xianjin YE Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_utils.py | 7 +++++++ python/pyspark/sql/utils.py | 8 +++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_utils.py b/python/pyspark/sql/tests/test_utils.py index fdadd5631cb73..072ea08085fba 100644 --- a/python/pyspark/sql/tests/test_utils.py +++ b/python/pyspark/sql/tests/test_utils.py @@ -1,3 +1,4 @@ +# -*- encoding: utf-8 -*- # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -26,6 +27,12 @@ def test_capture_analysis_exception(self): self.assertRaises(AnalysisException, lambda: self.spark.sql("select abc")) self.assertRaises(AnalysisException, lambda: self.df.selectExpr("a + b")) + def test_capture_user_friendly_exception(self): + try: + self.spark.sql("select `中文字段`") + except AnalysisException as e: + self.assertRegexpMatches(str(e), "cannot resolve '`中文字段`'") + def test_capture_parse_exception(self): self.assertRaises(ParseException, lambda: self.spark.sql("abc")) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index c30cc1482750a..996b7dd59ce9f 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -16,6 +16,7 @@ # import py4j +import sys class CapturedException(Exception): @@ -25,7 +26,12 @@ def __init__(self, desc, stackTrace, cause=None): self.cause = convert_exception(cause) if cause is not None else None def __str__(self): - return repr(self.desc) + desc = self.desc + # encode unicode instance for python2 for human readable description + if sys.version_info.major < 3 and isinstance(desc, unicode): + return str(desc.encode('utf-8')) + else: + return str(desc) class AnalysisException(CapturedException): From 376e17c082f0cb668a51f9b358ae66eace28ea9f Mon Sep 17 00:00:00 2001 From: sandeep katta Date: Wed, 18 Sep 2019 23:33:13 +0900 Subject: [PATCH 37/87] [SPARK-29101][SQL] Fix count API for csv file when DROPMALFORMED mode is selected ### What changes were proposed in this pull request? #DataSet fruit,color,price,quantity apple,red,1,3 banana,yellow,2,4 orange,orange,3,5 xxx This PR aims to fix the below ``` scala> spark.conf.set("spark.sql.csv.parser.columnPruning.enabled", false) scala> spark.read.option("header", "true").option("mode", "DROPMALFORMED").csv("fruit.csv").count res1: Long = 4 ``` This is caused by the issue [SPARK-24645](https://issues.apache.org/jira/browse/SPARK-24645). SPARK-24645 issue can also be solved by [SPARK-25387](https://issues.apache.org/jira/browse/SPARK-25387) ### Why are the changes needed? SPARK-24645 caused this regression, so reverted the code as it can also be solved by SPARK-25387 ### Does this PR introduce any user-facing change? No, ### How was this patch tested? Added UT, and also tested the bug SPARK-24645 **SPARK-24645 regression** ![image](https://user-images.githubusercontent.com/35216143/65067957-4c08ff00-d9a5-11e9-8d43-a4a23a61e8b8.png) Closes #25820 from sandeep-katta/SPARK-29101. Authored-by: sandeep katta Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/csv/UnivocityParser.scala | 7 ++++--- .../src/test/resources/test-data/malformedRow.csv | 5 +++++ .../sql/execution/datasources/csv/CSVSuite.scala | 14 ++++++++++++++ 3 files changed, 23 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/resources/test-data/malformedRow.csv diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 8456b7d218ead..39a08ec06c6a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -188,12 +188,13 @@ class UnivocityParser( } } - private val doParse = if (requiredSchema.nonEmpty) { - (input: String) => convert(tokenizer.parseLine(input)) - } else { + private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. (_: String) => InternalRow.empty + } else { + // parse if the columnPruning is disabled or requiredSchema is nonEmpty + (input: String) => convert(tokenizer.parseLine(input)) } /** diff --git a/sql/core/src/test/resources/test-data/malformedRow.csv b/sql/core/src/test/resources/test-data/malformedRow.csv new file mode 100644 index 0000000000000..8cfb3eefb982c --- /dev/null +++ b/sql/core/src/test/resources/test-data/malformedRow.csv @@ -0,0 +1,5 @@ +fruit,color,price,quantity +apple,red,1,3 +banana,yellow,2,4 +orange,orange,3,5 +malformedrow 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 5afd019c11a16..835fa4ef166b2 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 @@ -66,6 +66,7 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { private val unescapedQuotesFile = "test-data/unescaped-quotes.csv" private val valueMalformedFile = "test-data/value-malformed.csv" private val badAfterGoodFile = "test-data/bad_after_good.csv" + private val malformedRowFile = "test-data/malformedRow.csv" /** Verifies data and schema. */ private def verifyCars( @@ -2109,4 +2110,17 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { "expect the TextParsingException truncate the error content to be 1000 length.") } } + + test("SPARK-29101 test count with DROPMALFORMED mode") { + Seq((true, 4), (false, 3)).foreach { case (csvColumnPruning, expectedCount) => + withSQLConf(SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> csvColumnPruning.toString) { + val count = spark.read + .option("header", "true") + .option("mode", "DROPMALFORMED") + .csv(testFile(malformedRowFile)) + .count() + assert(expectedCount == count) + } + } + } } From d74fc6bb82f5a7cb8047955bd5fad10b11125e43 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 18 Sep 2019 09:41:02 -0500 Subject: [PATCH 38/87] [SPARK-29118][ML] Avoid redundant computation in transform of GMM & GLR ### What changes were proposed in this pull request? 1,GMM: obtaining the prediction (double) from its probabilty prediction(vector) 2,GLR: obtaining the prediction (double) from its link prediction(double) ### Why are the changes needed? it avoid predict twice ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #25815 from zhengruifeng/gmm_transform_opt. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../spark/ml/clustering/GaussianMixture.scala | 36 ++++++++++--------- .../GeneralizedLinearRegression.scala | 34 +++++++++--------- 2 files changed, 38 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 86caa1247e77f..979eb5e5448a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -33,8 +33,8 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions.udf +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.storage.StorageLevel @@ -111,28 +111,32 @@ class GaussianMixtureModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - var predictionColNames = Seq.empty[String] - var predictionColumns = Seq.empty[Column] - - if ($(predictionCol).nonEmpty) { - val predUDF = udf((vector: Vector) => predict(vector)) - predictionColNames :+= $(predictionCol) - predictionColumns :+= predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) - } + val vectorCol = DatasetUtils.columnToVector(dataset, $(featuresCol)) + var outputData = dataset + var numColsOutput = 0 if ($(probabilityCol).nonEmpty) { val probUDF = udf((vector: Vector) => predictProbability(vector)) - predictionColNames :+= $(probabilityCol) - predictionColumns :+= probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + outputData = outputData.withColumn($(probabilityCol), probUDF(vectorCol)) + numColsOutput += 1 + } + + if ($(predictionCol).nonEmpty) { + if ($(probabilityCol).nonEmpty) { + val predUDF = udf((vector: Vector) => vector.argmax) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(probabilityCol)))) + } else { + val predUDF = udf((vector: Vector) => predict(vector)) + outputData = outputData.withColumn($(predictionCol), predUDF(vectorCol)) + } + numColsOutput += 1 } - if (predictionColNames.nonEmpty) { - dataset.withColumns(predictionColNames, predictionColumns) - } else { + if (numColsOutput == 0) { this.logWarning(s"$uid: GaussianMixtureModel.transform() does nothing" + " because no output columns were set.") - dataset.toDF() } + outputData.toDF } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index a226ca49e6deb..4dc0c247ce331 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1036,31 +1036,33 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - var predictionColNames = Seq.empty[String] - var predictionColumns = Seq.empty[Column] - val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) + var outputData = dataset + var numColsOutput = 0 - if ($(predictionCol).nonEmpty) { - val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } - predictionColNames :+= $(predictionCol) - predictionColumns :+= predictUDF(col($(featuresCol)), offset) + if (hasLinkPredictionCol) { + val predLinkUDF = udf((features: Vector, offset: Double) => predictLink(features, offset)) + outputData = outputData + .withColumn($(linkPredictionCol), predLinkUDF(col($(featuresCol)), offset)) + numColsOutput += 1 } - if (hasLinkPredictionCol) { - val predictLinkUDF = - udf { (features: Vector, offset: Double) => predictLink(features, offset) } - predictionColNames :+= $(linkPredictionCol) - predictionColumns :+= predictLinkUDF(col($(featuresCol)), offset) + if ($(predictionCol).nonEmpty) { + if (hasLinkPredictionCol) { + val predUDF = udf((eta: Double) => familyAndLink.fitted(eta)) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(linkPredictionCol)))) + } else { + val predUDF = udf((features: Vector, offset: Double) => predict(features, offset)) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(featuresCol)), offset)) + } + numColsOutput += 1 } - if (predictionColNames.nonEmpty) { - dataset.withColumns(predictionColNames, predictionColumns) - } else { + if (numColsOutput == 0) { this.logWarning(s"$uid: GeneralizedLinearRegressionModel.transform() does nothing" + " because no output columns were set.") - dataset.toDF() } + outputData.toDF } /** From 276aaaae8d404975f8701089e9f4dfecd16e0d9f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Sep 2019 09:11:55 -0700 Subject: [PATCH 39/87] [SPARK-29105][CORE] Keep driver log file size up to date in HDFS HDFS doesn't update the file size reported by the NM if you just keep writing to the file; this makes the SHS believe the file is inactive, and so it may delete it after the configured max age for log files. This change uses hsync to keep the log file as up to date as possible when using HDFS. It also disables erasure coding by default for these logs, since hsync (& friends) does not work with EC. Tested with a SHS configured to aggressively clean up logs; verified a spark-shell session kept updating the log, which was not deleted by the SHS. Closes #25819 from vanzin/SPARK-29105. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../apache/spark/deploy/SparkHadoopUtil.scala | 54 ++++++++++--------- .../spark/internal/config/package.scala | 5 ++ .../scheduler/EventLoggingListener.scala | 8 +-- .../spark/util/logging/DriverLogger.scala | 16 +++++- docs/configuration.md | 10 ++++ .../spark/streaming/util/HdfsUtils.scala | 2 +- 6 files changed, 61 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 11420bb985520..8ba1ebf92a05a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -454,35 +454,39 @@ private[spark] object SparkHadoopUtil { // scalastyle:off line.size.limit /** - * Create a path that uses replication instead of erasure coding (ec), regardless of the default - * configuration in hdfs for the given path. This can be helpful as hdfs ec doesn't support - * hflush(), hsync(), or append() + * Create a file on the given file system, optionally making sure erasure coding is disabled. + * + * Disabling EC can be helpful as HDFS EC doesn't support hflush(), hsync(), or append(). * https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations */ // scalastyle:on line.size.limit - def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = { - try { - // Use reflection as this uses APIs only available in Hadoop 3 - val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) - // the builder api does not resolve relative paths, nor does it create parent dirs, while - // the old api does. - if (!fs.mkdirs(path.getParent())) { - throw new IOException(s"Failed to create parents of $path") + def createFile(fs: FileSystem, path: Path, allowEC: Boolean): FSDataOutputStream = { + if (allowEC) { + fs.create(path) + } else { + try { + // Use reflection as this uses APIs only available in Hadoop 3 + val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) + // the builder api does not resolve relative paths, nor does it create parent dirs, while + // the old api does. + if (!fs.mkdirs(path.getParent())) { + throw new IOException(s"Failed to create parents of $path") + } + val qualifiedPath = fs.makeQualified(path) + val builder = builderMethod.invoke(fs, qualifiedPath) + val builderCls = builder.getClass() + // this may throw a NoSuchMethodException if the path is not on hdfs + val replicateMethod = builderCls.getMethod("replicate") + val buildMethod = builderCls.getMethod("build") + val b2 = replicateMethod.invoke(builder) + buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream] + } catch { + case _: NoSuchMethodException => + // No createFile() method, we're using an older hdfs client, which doesn't give us control + // over EC vs. replication. Older hdfs doesn't have EC anyway, so just create a file with + // old apis. + fs.create(path) } - val qualifiedPath = fs.makeQualified(path) - val builder = builderMethod.invoke(fs, qualifiedPath) - val builderCls = builder.getClass() - // this may throw a NoSuchMethodException if the path is not on hdfs - val replicateMethod = builderCls.getMethod("replicate") - val buildMethod = builderCls.getMethod("build") - val b2 = replicateMethod.invoke(builder) - buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream] - } catch { - case _: NoSuchMethodException => - // No createFile() method, we're using an older hdfs client, which doesn't give us control - // over EC vs. replication. Older hdfs doesn't have EC anyway, so just create a file with - // old apis. - fs.create(path) } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6011901d1d53d..828b70dfe92e9 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -106,6 +106,11 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val DRIVER_LOG_ALLOW_EC = + ConfigBuilder("spark.driver.log.allowErasureCoding") + .booleanConf + .createWithDefault(false) + private[spark] val EVENT_LOG_ENABLED = ConfigBuilder("spark.eventLog.enabled") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 48eb2da3015f8..bee784dcdfe33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -67,7 +67,6 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) - private val shouldAllowECLogs = sparkConf.get(EVENT_LOG_ALLOW_EC) private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt @@ -121,11 +120,8 @@ private[spark] class EventLoggingListener( if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { new FileOutputStream(uri.getPath) } else { - hadoopDataStream = Some(if (shouldAllowECLogs) { - fileSystem.create(path) - } else { - SparkHadoopUtil.createNonECFile(fileSystem, path) - }) + hadoopDataStream = Some( + SparkHadoopUtil.createFile(fileSystem, path, sparkConf.get(EVENT_LOG_ALLOW_EC))) hadoopDataStream.get } diff --git a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala index c4540433bce97..4c1b49762ace3 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala @@ -18,15 +18,18 @@ package org.apache.spark.util.logging import java.io._ +import java.util.EnumSet import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream import org.apache.log4j.{FileAppender => Log4jFileAppender, _} import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils @@ -111,7 +114,8 @@ private[spark] class DriverLogger(conf: SparkConf) extends Logging { + DriverLogger.DRIVER_LOG_FILE_SUFFIX).getAbsolutePath() try { inStream = new BufferedInputStream(new FileInputStream(localLogFile)) - outputStream = fileSystem.create(new Path(dfsLogFile), true) + outputStream = SparkHadoopUtil.createFile(fileSystem, new Path(dfsLogFile), + conf.get(DRIVER_LOG_ALLOW_EC)) fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS) } catch { case e: Exception => @@ -131,12 +135,20 @@ private[spark] class DriverLogger(conf: SparkConf) extends Logging { } try { var remaining = inStream.available() + val hadData = remaining > 0 while (remaining > 0) { val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE)) outputStream.write(tmpBuffer, 0, read) remaining -= read } - outputStream.hflush() + if (hadData) { + outputStream match { + case hdfsStream: HdfsDataOutputStream => + hdfsStream.hsync(EnumSet.allOf(classOf[HdfsDataOutputStream.SyncFlag])) + case other => + other.hflush() + } + } } catch { case e: Exception => logError("Failed writing driver logs to dfs", e) } diff --git a/docs/configuration.md b/docs/configuration.md index e0b7386ac9bcc..e4a1fd322952b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -411,6 +411,16 @@ of the most common options to set are: use the default layout. + + spark.driver.log.allowErasureCoding + false + + Whether to allow driver logs to use erasure coding. On HDFS, erasure coded files will not + update as quickly as regular replicated files, so they make take longer to reflect changes + written by the application. Note that even if this is true, Spark will still not force the + file to use erasure coding, it will simply use file system defaults. + + Apart from these, the following properties are also available, and may be useful in some situations: diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 8cb68b2be4ecf..e5df2508268b8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -40,7 +40,7 @@ private[streaming] object HdfsUtils { } } else { // we dont' want to use hdfs erasure coding, as that lacks support for append and hflush - SparkHadoopUtil.createNonECFile(dfs, dfsPath) + SparkHadoopUtil.createFile(dfs, dfsPath, false) } } stream From ee94b5d7019f8ec181d42e953cb8b5190186fe30 Mon Sep 17 00:00:00 2001 From: John Zhuge Date: Wed, 18 Sep 2019 09:27:11 -0700 Subject: [PATCH 40/87] [SPARK-29030][SQL] Simplify lookupV2Relation ## What changes were proposed in this pull request? Simplify the return type for `lookupV2Relation` which makes the 3 callers more straightforward. ## How was this patch tested? Existing unit tests. Closes #25735 from jzhuge/lookupv2relation. Authored-by: John Zhuge Signed-off-by: Burak Yavuz --- .../sql/catalyst/analysis/Analyzer.scala | 87 +++++++------------ .../catalog/CatalogV2Implicits.scala | 6 +- 2 files changed, 37 insertions(+), 56 deletions(-) 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 0a13a34720846..76e59fa785426 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 @@ -38,7 +38,7 @@ 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 -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -666,20 +666,13 @@ class Analyzer( object ResolveTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case u: UnresolvedRelation => - val v2TableOpt = lookupV2Relation(u.multipartIdentifier) match { - case scala.Left((_, _, tableOpt)) => tableOpt - case scala.Right(tableOpt) => tableOpt - } - v2TableOpt.map(DataSourceV2Relation.create).getOrElse(u) + lookupV2Relation(u.multipartIdentifier) + .getOrElse(u) case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => - val v2TableOpt = lookupV2Relation(u.multipartIdentifier) match { - case scala.Left((_, _, tableOpt)) => tableOpt - case scala.Right(tableOpt) => tableOpt - } - v2TableOpt.map(DataSourceV2Relation.create).map { v2Relation => - i.copy(table = v2Relation) - }.getOrElse(i) + lookupV2Relation(u.multipartIdentifier) + .map(v2Relation => i.copy(table = v2Relation)) + .getOrElse(i) } } @@ -963,26 +956,13 @@ class Analyzer( private def resolveV2Alter( tableName: Seq[String], changes: Seq[TableChange]): Option[AlterTable] = { - lookupV2Relation(tableName) match { - case scala.Left((v2Catalog, ident, tableOpt)) => - Some(AlterTable( - v2Catalog.asTableCatalog, - ident, - tableOpt.map(DataSourceV2Relation.create).getOrElse(UnresolvedRelation(tableName)), - changes - )) - case scala.Right(tableOpt) => - tableOpt.map { table => - AlterTable( - sessionCatalog.asTableCatalog, - Identifier.of(tableName.init.toArray, tableName.last), - DataSourceV2Relation.create(table), - changes - ) - } + lookupV2RelationAndCatalog(tableName).map { + case (relation, catalog, ident) => + AlterTable(catalog.asTableCatalog, ident, relation, changes) } } } + /** * Resolve DESCRIBE TABLE statements that use a DSv2 catalog. * @@ -2840,36 +2820,35 @@ class Analyzer( /** * Performs the lookup of DataSourceV2 Tables. The order of resolution is: - * 1. Check if this relation is a temporary table - * 2. Check if it has a catalog identifier. Here we try to load the table. If we find the table, - * we can return the table. The result returned by an explicit catalog will be returned on - * the Left projection of the Either. - * 3. Try resolving the relation using the V2SessionCatalog if that is defined. If the - * V2SessionCatalog returns a V1 table definition (UnresolvedTable), then we return a `None` - * on the right side so that we can fallback to the V1 code paths. - * The basic idea is, if a value is returned on the Left, it means a v2 catalog is defined and - * must be used to resolve the table. If a value is returned on the right, then we can try - * creating a V2 relation if a V2 Table is defined. If it isn't defined, then we should defer - * to V1 code paths. + * 1. Check if this relation is a temporary table. + * 2. Check if it has a catalog identifier. Here we try to load the table. + * If we find the table, return the v2 relation and catalog. + * 3. Try resolving the relation using the V2SessionCatalog if that is defined. + * If the V2SessionCatalog returns a V1 table definition, + * return `None` so that we can fallback to the V1 code paths. + * If the V2SessionCatalog returns a V2 table, return the v2 relation and V2SessionCatalog. */ - private def lookupV2Relation( - identifier: Seq[String] - ): Either[(CatalogPlugin, Identifier, Option[Table]), Option[Table]] = { - import org.apache.spark.sql.connector.catalog.CatalogV2Util._ - + private def lookupV2RelationAndCatalog( + identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = identifier match { - case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => - scala.Right(None) + case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => None case CatalogObjectIdentifier(Some(v2Catalog), ident) => - scala.Left((v2Catalog, ident, loadTable(v2Catalog, ident))) + CatalogV2Util.loadTable(v2Catalog, ident) match { + case Some(table) => Some((DataSourceV2Relation.create(table), v2Catalog, ident)) + case None => None + } case CatalogObjectIdentifier(None, ident) => - loadTable(catalogManager.v2SessionCatalog, ident) match { - case Some(_: V1Table) => scala.Right(None) - case other => scala.Right(other) + CatalogV2Util.loadTable(catalogManager.v2SessionCatalog, ident) match { + case Some(_: V1Table) => None + case Some(table) => + Some((DataSourceV2Relation.create(table), catalogManager.v2SessionCatalog, ident)) + case None => None } - case _ => scala.Right(None) + case _ => None } - } + + private def lookupV2Relation(identifier: Seq[String]): Option[DataSourceV2Relation] = + lookupV2RelationAndCatalog(identifier).map(_._1) } /** 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 7d0acceec37da..031e0586a5f0f 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 @@ -92,8 +92,10 @@ private[sql] object CatalogV2Implicits { } } - implicit class MultipartIdentifierHelper(namespace: Seq[String]) { - def quoted: String = namespace.map(quote).mkString(".") + implicit class MultipartIdentifierHelper(parts: Seq[String]) { + def quoted: String = parts.map(quote).mkString(".") + + def asIdentifier: Identifier = Identifier.of(parts.init.toArray, parts.last) } private def quote(part: String): String = { From dfb0a8bb048d43f8fd1fb05b1027bd2fc7438dbc Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Wed, 18 Sep 2019 09:32:43 -0700 Subject: [PATCH 41/87] [SPARK-28208][BUILD][SQL] Upgrade to ORC 1.5.6 including closing the ORC readers ## What changes were proposed in this pull request? It upgrades ORC from 1.5.5 to 1.5.6 and adds closes the ORC readers when they aren't used to create RecordReaders. ## How was this patch tested? The changed unit tests were run. Closes #25006 from omalley/spark-28208. Lead-authored-by: Owen O'Malley Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 6 ++--- dev/deps/spark-deps-hadoop-3.2 | 6 ++--- pom.xml | 2 +- .../datasources/orc/OrcFileFormat.scala | 1 + .../execution/datasources/orc/OrcUtils.scala | 2 ++ .../v2/orc/OrcPartitionReaderFactory.scala | 2 ++ .../datasources/orc/OrcQuerySuite.scala | 24 ++++++++++++++----- .../datasources/orc/OrcSourceSuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 1 + 9 files changed, 32 insertions(+), 13 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 96cc76d0f2abb..f305be976e593 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,9 +156,9 @@ okapi-shade-0.4.2.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.5-nohive.jar -orc-mapreduce-1.5.5-nohive.jar -orc-shims-1.5.5.jar +orc-core-1.5.6-nohive.jar +orc-mapreduce-1.5.6-nohive.jar +orc-shims-1.5.6.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index a3a5b51226462..ae6fc0a0bac8e 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -175,9 +175,9 @@ okhttp-2.7.5.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.5-nohive.jar -orc-mapreduce-1.5.5-nohive.jar -orc-shims-1.5.5.jar +orc-core-1.5.6-nohive.jar +orc-mapreduce-1.5.6-nohive.jar +orc-shims-1.5.6.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index f67918af4dc90..ec6107d292f4b 100644 --- a/pom.xml +++ b/pom.xml @@ -139,7 +139,7 @@ 2.3.0 10.12.1.1 1.10.1 - 1.5.5 + 1.5.6 nohive com.twitter 1.6.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index f7c12598da209..59ee63aeab269 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -184,6 +184,7 @@ class OrcFileFormat val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, requiredSchema, reader, conf) + reader.close() if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 12d4244e19812..f3c92f32a3694 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -64,6 +64,7 @@ object OrcUtils extends Logging { try { val reader = OrcFile.createReader(file, readerOptions) val schema = reader.getSchema + reader.close() if (schema.getFieldNames.size == 0) { None } else { @@ -162,6 +163,7 @@ object OrcUtils extends Logging { if (matchedOrcFields.size > 1) { // Need to fail if there is ambiguity, i.e. more than one field is matched. val matchedOrcFieldsString = matchedOrcFields.mkString("[", ", ", "]") + reader.close() throw new RuntimeException(s"""Found duplicate field(s) "$requiredFieldName": """ + s"$matchedOrcFieldsString in case-insensitive mode") } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 3d6206128cabd..7780ce45e7c69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -78,6 +78,7 @@ case class OrcPartitionReaderFactory( val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) + reader.close() if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader[InternalRow] @@ -123,6 +124,7 @@ case class OrcPartitionReaderFactory( val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) + reader.close() if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index edc1822887f9f..c334d0116ebbb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -194,7 +194,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("ZLIB" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("ZLIB" === reader.getCompressionKind.name) + reader.close() } // `compression` overrides `orc.compress`. @@ -209,7 +211,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("ZLIB" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("ZLIB" === reader.getCompressionKind.name) + reader.close() } } @@ -225,7 +229,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("ZLIB" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("ZLIB" === reader.getCompressionKind.name) + reader.close() } withTempPath { file => @@ -238,7 +244,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("SNAPPY" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("SNAPPY" === reader.getCompressionKind.name) + reader.close() } withTempPath { file => @@ -251,7 +259,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("NONE" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("NONE" === reader.getCompressionKind.name) + reader.close() } } @@ -635,7 +645,9 @@ class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("LZO" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("LZO" === reader.getCompressionKind.name) + reader.close() } } 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 0d904a09c07e8..70d15bb195d53 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 @@ -374,6 +374,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { val readerOptions = OrcFile.readerOptions(new Configuration()) val reader = OrcFile.createReader(orcFilePath, readerOptions) val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString + reader.close() assert(version === SPARK_VERSION_SHORT) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index d63d741943bc7..487e888d588de 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2370,6 +2370,7 @@ class HiveDDLSuite assert(reader.getCompressionKind.name === "ZLIB") assert(reader.getCompressionSize == 1001) assert(reader.getRowIndexStride == 2002) + reader.close() } } } From cd481773c3577da5df3e1c0eb3b0a8c3c327684a Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Wed, 18 Sep 2019 10:32:10 -0700 Subject: [PATCH 42/87] [SPARK-28091][CORE] Extend Spark metrics system with user-defined metrics using executor plugins MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? This proposes to improve Spark instrumentation by adding a hook for user-defined metrics, extending Spark’s Dropwizard/Codahale metrics system. The original motivation of this work was to add instrumentation for S3 filesystem access metrics by Spark job. Currently, [[ExecutorSource]] instruments HDFS and local filesystem metrics. Rather than extending the code there, we proposes with this JIRA to add a metrics plugin system which is of more flexible and general use. Context: The Spark metrics system provides a large variety of metrics, see also , useful to monitor and troubleshoot Spark workloads. A typical workflow is to sink the metrics to a storage system and build dashboards on top of that. Highlights: - The metric plugin system makes it easy to implement instrumentation for S3 access by Spark jobs. - The metrics plugin system allows for easy extensions of how Spark collects HDFS-related workload metrics. This is currently done using the Hadoop Filesystem GetAllStatistics method, which is deprecated in recent versions of Hadoop. Recent versions of Hadoop Filesystem recommend using method GetGlobalStorageStatistics, which also provides several additional metrics. GetGlobalStorageStatistics is not available in Hadoop 2.7 (had been introduced in Hadoop 2.8). Using a metric plugin for Spark would allow an easy way to “opt in” using such new API calls for those deploying suitable Hadoop versions. - We also have the use case of adding Hadoop filesystem monitoring for a custom Hadoop compliant filesystem in use in our organization (EOS using the XRootD protocol). The metrics plugin infrastructure makes this easy to do. Others may have similar use cases. - More generally, this method makes it straightforward to plug in Filesystem and other metrics to the Spark monitoring system. Future work on plugin implementation can address extending monitoring to measure usage of external resources (OS, filesystem, network, accelerator cards, etc), that maybe would not normally be considered general enough for inclusion in Apache Spark code, but that can be nevertheless useful for specialized use cases, tests or troubleshooting. Implementation: The proposed implementation extends and modifies the work on Executor Plugin of SPARK-24918. Additionally, this is related to recent work on extending Spark executor metrics, such as SPARK-25228. As discussed during the review, the implementaiton of this feature modifies the Developer API for Executor Plugins, such that the new version is incompatible with the original version in Spark 2.4. ## How was this patch tested? This modifies existing tests for ExecutorPluginSuite to adapt them to the API changes. In addition, the new funtionality for registering pluginMetrics has been manually tested running Spark on YARN and K8S clusters, in particular for monitoring S3 and for extending HDFS instrumentation with the Hadoop Filesystem “GetGlobalStorageStatistics” metrics. Executor metric plugin example and code used for testing are available, for example at: https://github.com/cerndb/SparkExecutorPlugins Closes #24901 from LucaCanali/executorMetricsPlugin. Authored-by: Luca Canali Signed-off-by: Marcelo Vanzin --- .../java/org/apache/spark/ExecutorPlugin.java | 7 ++- .../apache/spark/ExecutorPluginContext.java | 50 +++++++++++++++++++ .../org/apache/spark/executor/Executor.scala | 10 +++- .../spark/executor/ExecutorPluginSource.scala | 30 +++++++++++ .../org/apache/spark/ExecutorPluginSuite.java | 46 +++++++++++++++-- docs/monitoring.md | 5 ++ project/MimaExcludes.scala | 3 ++ 7 files changed, 144 insertions(+), 7 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/ExecutorPluginContext.java create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java index f86520c81df33..b25c46266247e 100644 --- a/core/src/main/java/org/apache/spark/ExecutorPlugin.java +++ b/core/src/main/java/org/apache/spark/ExecutorPlugin.java @@ -40,12 +40,15 @@ public interface ExecutorPlugin { * Initialize the executor plugin. * *

    Each executor will, during its initialization, invoke this method on each - * plugin provided in the spark.executor.plugins configuration.

    + * plugin provided in the spark.executor.plugins configuration. The Spark executor + * will wait on the completion of the execution of the init method.

    * *

    Plugins should create threads in their implementation of this method for * any polling, blocking, or intensive computation.

    + * + * @param pluginContext Context information for the executor where the plugin is running. */ - default void init() {} + default void init(ExecutorPluginContext pluginContext) {} /** * Clean up and terminate this plugin. diff --git a/core/src/main/java/org/apache/spark/ExecutorPluginContext.java b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java new file mode 100644 index 0000000000000..8f018732b8217 --- /dev/null +++ b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import com.codahale.metrics.MetricRegistry; +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.annotation.Private; + +/** + * Encapsulates information about the executor when initializing {@link ExecutorPlugin} instances. + */ +@DeveloperApi +public class ExecutorPluginContext { + + public final MetricRegistry metricRegistry; + public final SparkConf sparkConf; + public final String executorId; + public final String executorHostName; + public final boolean isLocal; + + @Private + public ExecutorPluginContext( + MetricRegistry registry, + SparkConf conf, + String id, + String hostName, + boolean local) { + metricRegistry = registry; + sparkConf = conf; + executorId = id; + executorHostName = hostName; + isLocal = local; + } + +} diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a32226581b973..12bc227b341a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -146,8 +146,14 @@ private[spark] class Executor( Utils.withContextClassLoader(replClassLoader) { val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) plugins.foreach { plugin => - plugin.init() - logInfo(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + val pluginSource = new ExecutorPluginSource(plugin.getClass().getSimpleName()) + val pluginContext = new ExecutorPluginContext(pluginSource.metricRegistry, conf, + executorId, executorHostname, isLocal) + plugin.init(pluginContext) + logInfo("Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + if (pluginSource.metricRegistry.getNames.size() > 0) { + env.metricsSystem.registerSource(pluginSource) + } } plugins } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala new file mode 100644 index 0000000000000..5625e953c5e67 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import com.codahale.metrics.MetricRegistry + +import org.apache.spark.metrics.source.Source + +private[spark] +class ExecutorPluginSource(name: String) extends Source { + + override val metricRegistry = new MetricRegistry() + + override val sourceName = name +} diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java index 80cd70282a51d..ade13f02bde73 100644 --- a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java +++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java @@ -17,6 +17,8 @@ package org.apache.spark; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.MetricRegistry; import org.apache.spark.api.java.JavaSparkContext; import org.junit.After; @@ -30,6 +32,7 @@ public class ExecutorPluginSuite { private static final String testBadPluginName = TestBadShutdownPlugin.class.getName(); private static final String testPluginName = TestExecutorPlugin.class.getName(); private static final String testSecondPluginName = TestSecondPlugin.class.getName(); + private static final String testMetricsPluginName = TestMetricsPlugin.class.getName(); // Static value modified by testing plugins to ensure plugins loaded correctly. public static int numSuccessfulPlugins = 0; @@ -37,6 +40,10 @@ public class ExecutorPluginSuite { // Static value modified by testing plugins to verify plugins shut down properly. public static int numSuccessfulTerminations = 0; + // Static values modified by testing plugins to ensure metrics have been registered correctly. + public static MetricRegistry testMetricRegistry; + public static String gaugeName; + private JavaSparkContext sc; @Before @@ -107,8 +114,21 @@ public void testPluginShutdownWithException() { assertEquals(2, numSuccessfulTerminations); } + @Test + public void testPluginMetrics() { + // Verify that a custom metric is registered with the Spark metrics system + gaugeName = "test42"; + SparkConf conf = initializeSparkConf(testMetricsPluginName); + sc = new JavaSparkContext(conf); + assertEquals(1, numSuccessfulPlugins); + assertEquals(gaugeName, testMetricRegistry.getGauges().firstKey()); + sc.stop(); + sc = null; + assertEquals(1, numSuccessfulTerminations); + } + public static class TestExecutorPlugin implements ExecutorPlugin { - public void init() { + public void init(ExecutorPluginContext pluginContext) { ExecutorPluginSuite.numSuccessfulPlugins++; } @@ -118,7 +138,7 @@ public void shutdown() { } public static class TestSecondPlugin implements ExecutorPlugin { - public void init() { + public void init(ExecutorPluginContext pluginContext) { ExecutorPluginSuite.numSuccessfulPlugins++; } @@ -128,7 +148,7 @@ public void shutdown() { } public static class TestBadShutdownPlugin implements ExecutorPlugin { - public void init() { + public void init(ExecutorPluginContext pluginContext) { ExecutorPluginSuite.numSuccessfulPlugins++; } @@ -136,4 +156,24 @@ public void shutdown() { throw new RuntimeException("This plugin will fail to cleanly shut down"); } } + + public static class TestMetricsPlugin implements ExecutorPlugin { + public void init(ExecutorPluginContext myContext) { + MetricRegistry metricRegistry = myContext.metricRegistry; + // Registers a dummy metrics gauge for testing + String gaugeName = ExecutorPluginSuite.gaugeName; + metricRegistry.register(MetricRegistry.name(gaugeName), new Gauge() { + @Override + public Integer getValue() { + return 42; + } + }); + ExecutorPluginSuite.testMetricRegistry = metricRegistry; + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 8c81916d4f7d0..a45a41dc78cc3 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -1059,6 +1059,11 @@ when running in local mode. - hiveClientCalls.count - sourceCodeSize (histogram) +- namespace= + - Optional namespace(s). Metrics in this namespace are defined by user-supplied code, and + configured using the Spark executor plugin infrastructure. + See also the configuration parameter `spark.executor.plugins` + ### Source = JVM Source Notes: - Activate this source by setting the relevant `metrics.properties` file entry or the diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 097f1d2c2a6e1..38738112849f7 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,9 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-28091[CORE] Extend Spark metrics system with user-defined metrics using executor plugins + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExecutorPlugin.init"), + // [SPARK-][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createExternalTable"), From b4b2e958ce2d7142eb4ffb56401acf1739039555 Mon Sep 17 00:00:00 2001 From: bartosz25 Date: Wed, 18 Sep 2019 10:51:11 -0700 Subject: [PATCH 43/87] [MINOR][SS][DOCS] Adapt multiple watermark policy comment to the reality ### What changes were proposed in this pull request? Previous comment was true for Apache Spark 2.3.0. The 2.4.0 release brought multiple watermark policy and therefore stating that the 'min' is always chosen is misleading. This PR updates the comments about multiple watermark policy. They aren't true anymore since in case of multiple watermarks, we can configure which one will be applied to the query. This change was brought with Apache Spark 2.4.0 release. ### Why are the changes needed? It introduces some confusion about the real execution of the commented code. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? The tests weren't added because the change is only about the documentation level. I affirm that the contribution is my original work and that I license the work to the project under the project's open source license. Closes #25832 from bartosz25/fix_comments_multiple_watermark_policy. Authored-by: bartosz25 Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/streaming/WatermarkTracker.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala index 76ab1284633b1..b0f8cf9cd1846 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala @@ -63,7 +63,7 @@ case object MinWatermark extends MultipleWatermarkPolicy { } /** - * Policy to choose the *min* of the operator watermark values as the global watermark value. So the + * Policy to choose the *max* of the operator watermark values as the global watermark value. So the * global watermark will advance if any of the individual operator watermarks has advanced. * In other words, in a streaming query with multiple input streams and watermarks defined on all * of them, the global watermark will advance as fast as the fastest input. So if there is watermark @@ -108,10 +108,9 @@ case class WatermarkTracker(policy: MultipleWatermarkPolicy) extends Logging { } } - // Update the global watermark to the minimum of all watermark nodes. - // This is the safest option, because only the global watermark is fault-tolerant. Making - // it the minimum of all individual watermarks guarantees it will never advance past where - // any individual watermark operator would be if it were in a plan by itself. + // Update the global watermark accordingly to the chosen policy. To find all available policies + // and their semantics, please check the comments of + // `org.apache.spark.sql.execution.streaming.MultipleWatermarkPolicy` implementations. val chosenGlobalWatermark = policy.chooseGlobalWatermark(operatorToWatermarkMap.values.toSeq) if (chosenGlobalWatermark > globalWatermarkMs) { logInfo(s"Updating event-time watermark from $globalWatermarkMs to $chosenGlobalWatermark ms") From db9e0fda6bdfc3946a5011772b6b0180596ea331 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 18 Sep 2019 12:16:06 -0700 Subject: [PATCH 44/87] [SPARK-22796][PYTHON][ML] Add multiple columns support to PySpark QuantileDiscretizer ### What changes were proposed in this pull request? Add multiple columns support to PySpark QuantileDiscretizer ### Why are the changes needed? Multiple columns support for QuantileDiscretizer was in scala side a while ago. We need to add multiple columns support to python too. ### Does this PR introduce any user-facing change? Yes. New Python is added ### How was this patch tested? Add doctest Closes #25812 from huaxingao/spark-22796. Authored-by: Huaxin Gao Signed-off-by: Liang-Chi Hsieh --- python/pyspark/ml/feature.py | 121 +++++++++++++++++++++++++++-------- 1 file changed, 95 insertions(+), 26 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 5f55e48ce9f45..94bdd24f0f887 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1959,17 +1959,22 @@ def getDegree(self): @inherit_doc -class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInvalid, - JavaMLReadable, JavaMLWritable): +class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, + HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ - `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - categorical features. The number of bins can be set using the :py:attr:`numBuckets` parameter. - It is possible that the number of buckets used will be less than this value, for example, if - there are too few distinct values of the input to create enough distinct quantiles. + :py:class:`QuantileDiscretizer` takes a column with continuous features and outputs a column + with binned categorical features. The number of bins can be set using the :py:attr:`numBuckets` + parameter. It is possible that the number of buckets used will be less than this value, for + example, if there are too few distinct values of the input to create enough distinct quantiles. + Since 3.0.0, :py:class:`QuantileDiscretizer` can map multiple columns at once by setting the + :py:attr:`inputCols` parameter. If both of the :py:attr:`inputCol` and :py:attr:`inputCols` + parameters are set, an Exception will be thrown. To specify the number of buckets for each + column, the :py:attr:`numBucketsArray` parameter can be set, or if the number of buckets + should be the same across columns, :py:attr:`numBuckets` can be set as a convenience. NaN handling: Note also that - QuantileDiscretizer will raise an error when it finds NaN values in the dataset, but the user - can also choose to either keep or remove NaN values within the dataset by setting + :py:class:`QuantileDiscretizer` will raise an error when it finds NaN values in the dataset, + but the user can also choose to either keep or remove NaN values within the dataset by setting :py:attr:`handleInvalid` parameter. If the user chooses to keep NaN values, they will be handled specially and placed into their own bucket, for example, if 4 buckets are used, then non-NaN data will be put into buckets[0-3], but NaNs will be counted in a special bucket[4]. @@ -1981,29 +1986,61 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInv The lower and upper bin bounds will be `-Infinity` and `+Infinity`, covering all real values. >>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)] - >>> df = spark.createDataFrame(values, ["values"]) - >>> qds = QuantileDiscretizer(numBuckets=2, + >>> df1 = spark.createDataFrame(values, ["values"]) + >>> qds1 = QuantileDiscretizer(numBuckets=2, ... inputCol="values", outputCol="buckets", relativeError=0.01, handleInvalid="error") - >>> qds.getRelativeError() + >>> qds1.getRelativeError() 0.01 - >>> bucketizer = qds.fit(df) - >>> qds.setHandleInvalid("keep").fit(df).transform(df).count() + >>> bucketizer = qds1.fit(df1) + >>> qds1.setHandleInvalid("keep").fit(df1).transform(df1).count() 6 - >>> qds.setHandleInvalid("skip").fit(df).transform(df).count() + >>> qds1.setHandleInvalid("skip").fit(df1).transform(df1).count() 4 >>> splits = bucketizer.getSplits() >>> splits[0] -inf >>> print("%2.1f" % round(splits[1], 1)) 0.4 - >>> bucketed = bucketizer.transform(df).head() + >>> bucketed = bucketizer.transform(df1).head() >>> bucketed.buckets 0.0 >>> quantileDiscretizerPath = temp_path + "/quantile-discretizer" - >>> qds.save(quantileDiscretizerPath) + >>> qds1.save(quantileDiscretizerPath) >>> loadedQds = QuantileDiscretizer.load(quantileDiscretizerPath) - >>> loadedQds.getNumBuckets() == qds.getNumBuckets() + >>> loadedQds.getNumBuckets() == qds1.getNumBuckets() True + >>> inputs = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, 1.5), + ... (float("nan"), float("nan")), (float("nan"), float("nan"))] + >>> df2 = spark.createDataFrame(inputs, ["input1", "input2"]) + >>> qds2 = QuantileDiscretizer(relativeError=0.01, handleInvalid="error", numBuckets=2, + ... inputCols=["input1", "input2"], outputCols=["output1", "output2"]) + >>> qds2.getRelativeError() + 0.01 + >>> qds2.setHandleInvalid("keep").fit(df2).transform(df2).show() + +------+------+-------+-------+ + |input1|input2|output1|output2| + +------+------+-------+-------+ + | 0.1| 0.0| 0.0| 0.0| + | 0.4| 1.0| 1.0| 1.0| + | 1.2| 1.3| 1.0| 1.0| + | 1.5| 1.5| 1.0| 1.0| + | NaN| NaN| 2.0| 2.0| + | NaN| NaN| 2.0| 2.0| + +------+------+-------+-------+ + ... + >>> qds3 = QuantileDiscretizer(relativeError=0.01, handleInvalid="error", + ... numBucketsArray=[5, 10], inputCols=["input1", "input2"], + ... outputCols=["output1", "output2"]) + >>> qds3.setHandleInvalid("skip").fit(df2).transform(df2).show() + +------+------+-------+-------+ + |input1|input2|output1|output2| + +------+------+-------+-------+ + | 0.1| 0.0| 1.0| 1.0| + | 0.4| 1.0| 2.0| 2.0| + | 1.2| 1.3| 3.0| 3.0| + | 1.5| 1.5| 4.0| 4.0| + +------+------+-------+-------+ + ... .. versionadded:: 2.0.0 """ @@ -2021,15 +2058,26 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInv handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. " + "Options are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special " + - "additional bucket).", + "additional bucket). Note that in the multiple columns " + + "case, the invalid handling is applied to all columns. That said " + + "for 'error' it will throw an error if any invalids are found in " + + "any columns, for 'skip' it will skip rows with any invalids in " + + "any columns, etc.", typeConverter=TypeConverters.toString) + numBucketsArray = Param(Params._dummy(), "numBucketsArray", "Array of number of buckets " + + "(quantiles, or categories) into which data points are grouped. " + + "This is for multiple columns input. If transforming multiple " + + "columns and numBucketsArray is not set, but numBuckets is set, " + + "then numBuckets will be applied across all columns.", + typeConverter=TypeConverters.toListInt) + @keyword_only def __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, - handleInvalid="error"): + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None): """ __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, \ - handleInvalid="error") + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None) """ super(QuantileDiscretizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.QuantileDiscretizer", @@ -2041,10 +2089,10 @@ def __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0. @keyword_only @since("2.0.0") def setParams(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, - handleInvalid="error"): + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None): """ setParams(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, \ - handleInvalid="error") + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None) Set the params for the QuantileDiscretizer """ kwargs = self._input_kwargs @@ -2064,6 +2112,20 @@ def getNumBuckets(self): """ return self.getOrDefault(self.numBuckets) + @since("3.0.0") + def setNumBucketsArray(self, value): + """ + Sets the value of :py:attr:`numBucketsArray`. + """ + return self._set(numBucketsArray=value) + + @since("3.0.0") + def getNumBucketsArray(self): + """ + Gets the value of numBucketsArray or its default value. + """ + return self.getOrDefault(self.numBucketsArray) + @since("2.0.0") def setRelativeError(self, value): """ @@ -2082,10 +2144,17 @@ def _create_model(self, java_model): """ Private method to convert the java_model to a Python model. """ - return Bucketizer(splits=list(java_model.getSplits()), - inputCol=self.getInputCol(), - outputCol=self.getOutputCol(), - handleInvalid=self.getHandleInvalid()) + if (self.isSet(self.inputCol)): + return Bucketizer(splits=list(java_model.getSplits()), + inputCol=self.getInputCol(), + outputCol=self.getOutputCol(), + handleInvalid=self.getHandleInvalid()) + else: + splitsArrayList = [list(x) for x in list(java_model.getSplitsArray())] + return Bucketizer(splitsArray=splitsArrayList, + inputCols=self.getInputCols(), + outputCols=self.getOutputCols(), + handleInvalid=self.getHandleInvalid()) @inherit_doc From f32f16fd68f51a94d6adb2d01a9b2e557885e656 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Sep 2019 13:30:00 -0700 Subject: [PATCH 45/87] [SPARK-29082][CORE] Skip delegation token generation if no credentials are available This situation can happen when an external system (e.g. Oozie) generates delegation tokens for a Spark application. The Spark driver will then run against secured services, have proper credentials (the tokens), but no kerberos credentials. So trying to do things that requires a kerberos credential fails. Instead, if no kerberos credentials are detected, just skip the whole delegation token code. Tested with an application that simulates Oozie; fails before the fix, passes with the fix. Also with other DT-related tests to make sure other functionality keeps working. Closes #25805 from vanzin/SPARK-29082. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../HadoopDelegationTokenManager.scala | 22 +++++++---- .../CoarseGrainedSchedulerBackend.scala | 10 +++-- .../HadoopDelegationTokenManagerSuite.scala | 39 ++++++++++++++++++- 3 files changed, 59 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 759d857d56e0e..f769ce468e49c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -140,13 +140,21 @@ private[spark] class HadoopDelegationTokenManager( * @param creds Credentials object where to store the delegation tokens. */ def obtainDelegationTokens(creds: Credentials): Unit = { - val freshUGI = doLogin() - freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { - override def run(): Unit = { - val (newTokens, _) = obtainDelegationTokens() - creds.addAll(newTokens) - } - }) + val currentUser = UserGroupInformation.getCurrentUser() + val hasKerberosCreds = principal != null || + Option(currentUser.getRealUser()).getOrElse(currentUser).hasKerberosCredentials() + + // Delegation tokens can only be obtained if the real user has Kerberos credentials, so + // skip creation when those are not available. + if (hasKerberosCreds) { + val freshUGI = doLogin() + freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val (newTokens, _) = obtainDelegationTokens() + creds.addAll(newTokens) + } + }) + } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d81070c362ba6..c9b408bed1163 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -427,12 +427,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val ugi = UserGroupInformation.getCurrentUser() val tokens = if (dtm.renewalEnabled) { dtm.start() - } else if (ugi.hasKerberosCredentials() || SparkHadoopUtil.get.isProxyUser(ugi)) { + } else { val creds = ugi.getCredentials() dtm.obtainDelegationTokens(creds) - SparkHadoopUtil.get.serialize(creds) - } else { - null + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } } if (tokens != null) { updateDelegationTokens(tokens) diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index 70174f7ff939a..bf53386da6304 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.deploy.security +import java.security.PrivilegedExceptionAction + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.security.HadoopDelegationTokenProvider private class ExceptionThrowingDelegationTokenProvider extends HadoopDelegationTokenProvider { @@ -69,4 +73,37 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(!manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) } + + test("SPARK-29082: do not fail if current user does not have credentials") { + // SparkHadoopUtil overrides the UGI configuration during initialization. That normally + // happens early in the Spark application, but here it may affect the test depending on + // how it's run, so force its initialization. + SparkHadoopUtil.get + + val krbConf = new Configuration() + krbConf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos") + + UserGroupInformation.setConfiguration(krbConf) + try { + val manager = new HadoopDelegationTokenManager(new SparkConf(false), krbConf, null) + val testImpl = new PrivilegedExceptionAction[Unit] { + override def run(): Unit = { + assert(UserGroupInformation.isSecurityEnabled()) + val creds = new Credentials() + manager.obtainDelegationTokens(creds) + assert(creds.numberOfTokens() === 0) + assert(creds.numberOfSecretKeys() === 0) + } + } + + val realUser = UserGroupInformation.createUserForTesting("realUser", Array.empty) + realUser.doAs(testImpl) + + val proxyUser = UserGroupInformation.createProxyUserForTesting("proxyUser", realUser, + Array.empty) + proxyUser.doAs(testImpl) + } finally { + UserGroupInformation.reset() + } + } } From 8c3f27ceb43d7cfb78bda7f39e2758fb26f660d4 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 18 Sep 2019 13:30:36 -0700 Subject: [PATCH 46/87] [SPARK-28683][BUILD] Upgrade Scala to 2.12.10 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? This PR upgrade Scala to **2.12.10**. Release notes: - Fix regression in large string interpolations with non-String typed splices - Revert "Generate shallower ASTs in pattern translation" - Fix regression in classpath when JARs have 'a.b' entries beside 'a/b' - Faster compiler: 5–10% faster since 2.12.8 - Improved compatibility with JDK 11, 12, and 13 - Experimental support for build pipelining and outline type checking More details: https://github.com/scala/scala/releases/tag/v2.12.10 https://github.com/scala/scala/releases/tag/v2.12.9 ## How was this patch tested? Existing tests Closes #25404 from wangyum/SPARK-28683. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- dev/deps/spark-deps-hadoop-3.2 | 6 +++--- docs/_config.yml | 2 +- external/docker/spark-test/base/Dockerfile | 2 +- pom.xml | 2 +- project/SparkBuild.scala | 2 +- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 2 +- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f305be976e593..cae2614128b1a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -172,10 +172,10 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar -scala-compiler-2.12.8.jar -scala-library-2.12.8.jar +scala-compiler-2.12.10.jar +scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.8.jar +scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar shapeless_2.12-2.3.2.jar shims-0.7.45.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index ae6fc0a0bac8e..21df5a70e7f8d 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -191,10 +191,10 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar re2j-1.1.jar -scala-compiler-2.12.8.jar -scala-library-2.12.8.jar +scala-compiler-2.12.10.jar +scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.8.jar +scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar shapeless_2.12-2.3.2.jar shims-0.7.45.jar diff --git a/docs/_config.yml b/docs/_config.yml index 146c90fcff6e5..57b8d716ee55c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -17,7 +17,7 @@ include: SPARK_VERSION: 3.0.0-SNAPSHOT SPARK_VERSION_SHORT: 3.0.0 SCALA_BINARY_VERSION: "2.12" -SCALA_VERSION: "2.12.8" +SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/external/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile index c1fd630d0b665..5bec5d3f16548 100644 --- a/external/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -25,7 +25,7 @@ RUN apt-get update && \ apt-get install -y less openjdk-8-jre-headless iproute2 vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.12.8 +ENV SCALA_VERSION 2.12.10 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark diff --git a/pom.xml b/pom.xml index ec6107d292f4b..a78a5beb814f5 100644 --- a/pom.xml +++ b/pom.xml @@ -164,7 +164,7 @@ 3.4.1 3.2.2 - 2.12.8 + 2.12.10 2.12 --diff --test diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 25c2fb4af5c34..d984f5dacf1af 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -223,7 +223,7 @@ object SparkBuild extends PomBuild { .map(file), incOptions := incOptions.value.withNameHashing(true), publishMavenStyle := true, - unidocGenjavadocVersion := "0.13", + unidocGenjavadocVersion := "0.14", // Override SBT's default resolvers: resolvers := Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index fbbbda33538d1..ceb3155500385 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -72,7 +72,7 @@ object ScalaReflection extends ScalaReflection { /** * Synchronize to prevent concurrent usage of `<:<` operator. * This operator is not thread safe in any current version of scala; i.e. - * (2.11.12, 2.12.8, 2.13.0-M5). + * (2.11.12, 2.12.10, 2.13.0-M5). * * See https://github.com/scala/bug/issues/10766 */ From a6a663c4379390217443bc5b6f75873fb1c38c73 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Sep 2019 17:52:23 -0700 Subject: [PATCH 47/87] [SPARK-29141][SQL][TEST] Use SqlBasedBenchmark in SQL benchmarks ### What changes were proposed in this pull request? Refactored SQL-related benchmark and made them depend on `SqlBasedBenchmark`. In particular, creation of Spark session are moved into `override def getSparkSession: SparkSession`. ### Why are the changes needed? This should simplify maintenance of SQL-based benchmarks by reducing the number of dependencies. In the future, it should be easier to refactor & extend all SQL benchmarks by changing only one trait. Finally, all SQL-based benchmarks will look uniformly. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the modified benchmarks. Closes #25828 from MaxGekk/sql-benchmarks-refactoring. Lead-authored-by: Maxim Gekk Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../benchmarks/AvroReadBenchmark-results.txt | 124 ++--- .../benchmark/AvroReadBenchmark.scala | 3 +- .../DataSourceReadBenchmark-results.txt | 428 +++++++++--------- .../benchmarks/ExtractBenchmark-results.txt | 176 +++---- .../PrimitiveArrayBenchmark-results.txt | 13 +- .../benchmark/DataSourceReadBenchmark.scala | 38 +- .../benchmark/ExtractBenchmark.scala | 10 +- .../benchmark/FilterPushdownBenchmark.scala | 31 +- .../benchmark/PrimitiveArrayBenchmark.scala | 23 +- ...jectHashAggregateExecBenchmark-results.txt | 48 +- .../benchmarks/OrcReadBenchmark-results.txt | 192 ++++---- .../ObjectHashAggregateExecBenchmark.scala | 8 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 33 +- 13 files changed, 566 insertions(+), 561 deletions(-) diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt index 7900fea453b10..8a12d9d15d915 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -2,121 +2,121 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2774 / 2815 5.7 176.4 1.0X +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 -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 -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2761 / 2777 5.7 175.5 1.0X +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 -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 -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2783 / 2870 5.7 176.9 1.0X +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 -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 -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 3256 / 3266 4.8 207.0 1.0X +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 -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 -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2841 / 2867 5.5 180.6 1.0X +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 -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 -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2981 / 2996 5.3 189.5 1.0X +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 ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of columns 4781 / 4783 2.2 456.0 1.0X +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 ================================================================================================ Partitioned Table Scan ================================================================================================ -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 -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column 3372 / 3386 4.7 214.4 1.0X -Partition column 3035 / 3064 5.2 193.0 1.1X -Both columns 3445 / 3461 4.6 219.1 1.0X +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 ================================================================================================ Repeated String Scan ================================================================================================ -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 -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 3395 / 3401 3.1 323.8 1.0X +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 ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 5580 / 5624 1.9 532.2 1.0X +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 -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 -String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 4622 / 4623 2.3 440.8 1.0X +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 -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 -String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 3238 / 3241 3.2 308.8 1.0X +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 ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of single column 5472 / 5484 0.2 5218.8 1.0X +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 -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 -Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of single column 10680 / 10701 0.1 10185.1 1.0X +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 -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 -Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of single column 16143 / 16238 0.1 15394.9 1.0X +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 diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala index f2f7d650066fb..a16126ae24246 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -22,7 +22,6 @@ import scala.util.Random import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.types._ /** @@ -36,7 +35,7 @@ import org.apache.spark.sql.types._ * Results will be written to "benchmarks/AvroReadBenchmark-results.txt". * }}} */ -object AvroReadBenchmark extends SqlBasedBenchmark with SQLHelper { +object AvroReadBenchmark extends SqlBasedBenchmark { def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) } diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt index f547f61654b5f..efb0601cc5d65 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -2,251 +2,251 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 26366 / 26562 0.6 1676.3 1.0X -SQL Json 8709 / 8724 1.8 553.7 3.0X -SQL Parquet Vectorized 166 / 187 94.8 10.5 159.0X -SQL Parquet MR 1706 / 1720 9.2 108.4 15.5X -SQL ORC Vectorized 167 / 174 94.2 10.6 157.9X -SQL ORC MR 1433 / 1465 11.0 91.1 18.4X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 200 / 207 78.7 12.7 1.0X -ParquetReader Vectorized -> Row 117 / 119 134.7 7.4 1.7X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 26489 / 26547 0.6 1684.1 1.0X -SQL Json 8990 / 8998 1.7 571.5 2.9X -SQL Parquet Vectorized 209 / 221 75.1 13.3 126.5X -SQL Parquet MR 1949 / 1949 8.1 123.9 13.6X -SQL ORC Vectorized 221 / 228 71.3 14.0 120.1X -SQL ORC MR 1527 / 1549 10.3 97.1 17.3X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 286 / 296 54.9 18.2 1.0X -ParquetReader Vectorized -> Row 249 / 253 63.1 15.8 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 27701 / 27744 0.6 1761.2 1.0X -SQL Json 9703 / 9733 1.6 616.9 2.9X -SQL Parquet Vectorized 176 / 182 89.2 11.2 157.0X -SQL Parquet MR 2164 / 2173 7.3 137.6 12.8X -SQL ORC Vectorized 307 / 314 51.2 19.5 90.2X -SQL ORC MR 1690 / 1700 9.3 107.4 16.4X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 259 / 277 60.7 16.5 1.0X -ParquetReader Vectorized -> Row 261 / 265 60.3 16.6 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 34813 / 34900 0.5 2213.3 1.0X -SQL Json 12570 / 12617 1.3 799.2 2.8X -SQL Parquet Vectorized 270 / 308 58.2 17.2 128.9X -SQL Parquet MR 2427 / 2431 6.5 154.3 14.3X -SQL ORC Vectorized 388 / 398 40.6 24.6 89.8X -SQL ORC MR 1819 / 1851 8.6 115.7 19.1X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 372 / 379 42.3 23.7 1.0X -ParquetReader Vectorized -> Row 357 / 368 44.1 22.7 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 28753 / 28781 0.5 1828.0 1.0X -SQL Json 12039 / 12215 1.3 765.4 2.4X -SQL Parquet Vectorized 170 / 177 92.4 10.8 169.0X -SQL Parquet MR 2184 / 2196 7.2 138.9 13.2X -SQL ORC Vectorized 432 / 440 36.4 27.5 66.5X -SQL ORC MR 1812 / 1833 8.7 115.2 15.9X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 253 / 260 62.2 16.1 1.0X -ParquetReader Vectorized -> Row 256 / 257 61.6 16.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 36177 / 36188 0.4 2300.1 1.0X -SQL Json 18895 / 18898 0.8 1201.3 1.9X -SQL Parquet Vectorized 267 / 276 58.9 17.0 135.6X -SQL Parquet MR 2355 / 2363 6.7 149.7 15.4X -SQL ORC Vectorized 543 / 546 29.0 34.5 66.6X -SQL ORC MR 2246 / 2258 7.0 142.8 16.1X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 353 / 367 44.6 22.4 1.0X -ParquetReader Vectorized -> Row 351 / 357 44.7 22.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 +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 + +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 + +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 + +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 + +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 + +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 + +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 + +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 + +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 + +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 + +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 + +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 ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 21130 / 21246 0.5 2015.1 1.0X -SQL Json 12145 / 12174 0.9 1158.2 1.7X -SQL Parquet Vectorized 2363 / 2377 4.4 225.3 8.9X -SQL Parquet MR 4555 / 4557 2.3 434.4 4.6X -SQL ORC Vectorized 2361 / 2388 4.4 225.1 9.0X -SQL ORC MR 4186 / 4209 2.5 399.2 5.0X +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 ================================================================================================ Repeated String Scan ================================================================================================ -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 -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 11693 / 11729 0.9 1115.1 1.0X -SQL Json 7025 / 7025 1.5 669.9 1.7X -SQL Parquet Vectorized 803 / 821 13.1 76.6 14.6X -SQL Parquet MR 1776 / 1790 5.9 169.4 6.6X -SQL ORC Vectorized 491 / 494 21.4 46.8 23.8X -SQL ORC MR 2050 / 2063 5.1 195.5 5.7X +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 ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column - CSV 30965 / 31041 0.5 1968.7 1.0X -Data column - Json 12876 / 12882 1.2 818.6 2.4X -Data column - Parquet Vectorized 277 / 282 56.7 17.6 111.6X -Data column - Parquet MR 3398 / 3402 4.6 216.0 9.1X -Data column - ORC Vectorized 399 / 407 39.4 25.4 77.5X -Data column - ORC MR 2583 / 2589 6.1 164.2 12.0X -Partition column - CSV 7403 / 7427 2.1 470.7 4.2X -Partition column - Json 5587 / 5625 2.8 355.2 5.5X -Partition column - Parquet Vectorized 71 / 78 222.6 4.5 438.3X -Partition column - Parquet MR 1798 / 1808 8.7 114.3 17.2X -Partition column - ORC Vectorized 72 / 75 219.0 4.6 431.2X -Partition column - ORC MR 1772 / 1778 8.9 112.6 17.5X -Both columns - CSV 30211 / 30212 0.5 1920.7 1.0X -Both columns - Json 13382 / 13391 1.2 850.8 2.3X -Both columns - Parquet Vectorized 321 / 333 49.0 20.4 96.4X -Both columns - Parquet MR 3656 / 3661 4.3 232.4 8.5X -Both columns - ORC Vectorized 443 / 448 35.5 28.2 69.9X -Both columns - ORC MR 2626 / 2633 6.0 167.0 11.8X +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 ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13918 / 13979 0.8 1327.3 1.0X -SQL Json 10068 / 10068 1.0 960.1 1.4X -SQL Parquet Vectorized 1563 / 1564 6.7 149.0 8.9X -SQL Parquet MR 3835 / 3836 2.7 365.8 3.6X -ParquetReader Vectorized 1115 / 1118 9.4 106.4 12.5X -SQL ORC Vectorized 1172 / 1208 8.9 111.8 11.9X -SQL ORC MR 3708 / 3711 2.8 353.6 3.8X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13972 / 14043 0.8 1332.5 1.0X -SQL Json 7436 / 7469 1.4 709.1 1.9X -SQL Parquet Vectorized 1103 / 1112 9.5 105.2 12.7X -SQL Parquet MR 2841 / 2847 3.7 271.0 4.9X -ParquetReader Vectorized 992 / 1012 10.6 94.6 14.1X -SQL ORC Vectorized 1275 / 1349 8.2 121.6 11.0X -SQL ORC MR 3244 / 3259 3.2 309.3 4.3X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 11228 / 11244 0.9 1070.8 1.0X -SQL Json 5200 / 5247 2.0 495.9 2.2X -SQL Parquet Vectorized 238 / 242 44.1 22.7 47.2X -SQL Parquet MR 1730 / 1734 6.1 165.0 6.5X -ParquetReader Vectorized 237 / 238 44.3 22.6 47.4X -SQL ORC Vectorized 459 / 462 22.8 43.8 24.4X -SQL ORC MR 1767 / 1783 5.9 168.5 6.4X +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 + +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 + +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 ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 3322 / 3356 0.3 3167.9 1.0X -SQL Json 2808 / 2843 0.4 2678.2 1.2X -SQL Parquet Vectorized 56 / 63 18.9 52.9 59.8X -SQL Parquet MR 215 / 219 4.9 205.4 15.4X -SQL ORC Vectorized 64 / 76 16.4 60.9 52.0X -SQL ORC MR 314 / 316 3.3 299.6 10.6X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 50 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 7978 / 7989 0.1 7608.5 1.0X -SQL Json 10294 / 10325 0.1 9816.9 0.8X -SQL Parquet Vectorized 72 / 85 14.5 69.0 110.3X -SQL Parquet MR 237 / 241 4.4 226.4 33.6X -SQL ORC Vectorized 82 / 92 12.7 78.5 97.0X -SQL ORC MR 900 / 909 1.2 858.5 8.9X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13489 / 13508 0.1 12864.3 1.0X -SQL Json 18813 / 18827 0.1 17941.4 0.7X -SQL Parquet Vectorized 107 / 111 9.8 101.8 126.3X -SQL Parquet MR 275 / 286 3.8 262.3 49.0X -SQL ORC Vectorized 107 / 115 9.8 101.7 126.4X -SQL ORC MR 1659 / 1664 0.6 1582.3 8.1X +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 + +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 + +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 diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 31ad7878ee643..6c4bdfed703ff 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,100 +1,100 @@ -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 264 281 25 37.9 26.4 1.0X -MILLENNIUM of timestamp 1187 1198 16 8.4 118.7 0.2X -CENTURY of timestamp 1122 1124 2 8.9 112.2 0.2X -DECADE of timestamp 1041 1049 7 9.6 104.1 0.3X -YEAR of timestamp 1027 1035 6 9.7 102.7 0.3X -ISOYEAR of timestamp 1155 1171 27 8.7 115.5 0.2X -QUARTER of timestamp 1181 1192 10 8.5 118.1 0.2X -MONTH of timestamp 1023 1030 7 9.8 102.3 0.3X -WEEK of timestamp 1511 1517 5 6.6 151.1 0.2X -DAY of timestamp 1010 1016 6 9.9 101.0 0.3X -DAYOFWEEK of timestamp 1127 1129 4 8.9 112.7 0.2X -DOW of timestamp 1123 1130 6 8.9 112.3 0.2X -ISODOW of timestamp 1099 1105 6 9.1 109.9 0.2X -DOY of timestamp 1029 1030 1 9.7 102.9 0.3X -HOUR of timestamp 415 417 1 24.1 41.5 0.6X -MINUTE of timestamp 409 418 13 24.4 40.9 0.6X -SECOND of timestamp 408 413 8 24.5 40.8 0.6X -MILLISECONDS of timestamp 28956 29040 73 0.3 2895.6 0.0X -MICROSECONDS of timestamp 504 519 13 19.8 50.4 0.5X -EPOCH of timestamp 23543 23566 28 0.4 2354.3 0.0X +cast to timestamp 397 428 45 25.2 39.7 1.0X +MILLENNIUM of timestamp 1480 1546 67 6.8 148.0 0.3X +CENTURY of timestamp 1368 1384 17 7.3 136.8 0.3X +DECADE of timestamp 1281 1344 57 7.8 128.1 0.3X +YEAR of timestamp 1238 1244 5 8.1 123.8 0.3X +ISOYEAR of timestamp 1379 1455 122 7.2 137.9 0.3X +QUARTER of timestamp 1442 1456 15 6.9 144.2 0.3X +MONTH of timestamp 1213 1217 3 8.2 121.3 0.3X +WEEK of timestamp 1927 1947 22 5.2 192.7 0.2X +DAY of timestamp 1306 1320 16 7.7 130.6 0.3X +DAYOFWEEK of timestamp 1394 1402 11 7.2 139.4 0.3X +DOW of timestamp 1367 1374 6 7.3 136.7 0.3X +ISODOW of timestamp 1317 1321 5 7.6 131.7 0.3X +DOY of timestamp 1223 1238 14 8.2 122.3 0.3X +HOUR of timestamp 361 362 2 27.7 36.1 1.1X +MINUTE of timestamp 354 362 10 28.3 35.4 1.1X +SECOND of timestamp 362 365 4 27.6 36.2 1.1X +MILLISECONDS of timestamp 36723 36761 63 0.3 3672.3 0.0X +MICROSECONDS of timestamp 469 490 29 21.3 46.9 0.8X +EPOCH of timestamp 30137 30181 38 0.3 3013.7 0.0X -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 853 856 6 11.7 85.3 1.0X -MILLENNIUM of date 1081 1088 6 9.2 108.1 0.8X -CENTURY of date 1087 1090 4 9.2 108.7 0.8X -DECADE of date 1018 1019 1 9.8 101.8 0.8X -YEAR of date 996 1006 9 10.0 99.6 0.9X -ISOYEAR of date 1133 1147 21 8.8 113.3 0.8X -QUARTER of date 1246 1254 10 8.0 124.6 0.7X -MONTH of date 998 1002 4 10.0 99.8 0.9X -WEEK of date 1483 1490 7 6.7 148.3 0.6X -DAY of date 992 998 5 10.1 99.2 0.9X -DAYOFWEEK of date 1121 1128 7 8.9 112.1 0.8X -DOW of date 1118 1126 8 8.9 111.8 0.8X -ISODOW of date 1093 1103 9 9.1 109.3 0.8X -DOY of date 1026 1032 5 9.7 102.6 0.8X -HOUR of date 1707 1726 24 5.9 170.7 0.5X -MINUTE of date 1710 1731 19 5.8 171.0 0.5X -SECOND of date 1701 1720 19 5.9 170.1 0.5X -MILLISECONDS of date 2256 2272 19 4.4 225.6 0.4X -MICROSECONDS of date 1801 1810 11 5.6 180.1 0.5X -EPOCH of date 24848 24860 17 0.4 2484.8 0.0X +cast to date 1010 1022 11 9.9 101.0 1.0X +MILLENNIUM of date 1300 1311 18 7.7 130.0 0.8X +CENTURY of date 1304 1306 2 7.7 130.4 0.8X +DECADE of date 1199 1205 10 8.3 119.9 0.8X +YEAR of date 1191 1194 4 8.4 119.1 0.8X +ISOYEAR of date 1451 1456 9 6.9 145.1 0.7X +QUARTER of date 1494 1501 10 6.7 149.4 0.7X +MONTH of date 1189 1191 3 8.4 118.9 0.8X +WEEK of date 1893 1958 111 5.3 189.3 0.5X +DAY of date 1282 1285 3 7.8 128.2 0.8X +DAYOFWEEK of date 1374 1386 17 7.3 137.4 0.7X +DOW of date 1348 1351 3 7.4 134.8 0.7X +ISODOW of date 1292 1297 5 7.7 129.2 0.8X +DOY of date 1213 1216 3 8.2 121.3 0.8X +HOUR of date 1450 1458 9 6.9 145.0 0.7X +MINUTE of date 1445 1452 9 6.9 144.5 0.7X +SECOND of date 1448 1458 8 6.9 144.8 0.7X +MILLISECONDS of date 2094 2103 11 4.8 209.4 0.5X +MICROSECONDS of date 1562 1573 19 6.4 156.2 0.6X +EPOCH of date 31000 31047 68 0.3 3100.0 0.0X -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 227 253 39 44.0 22.7 1.0X -MILLENNIUM of timestamp 1121 1127 9 8.9 112.1 0.2X -CENTURY of timestamp 1084 1094 17 9.2 108.4 0.2X -DECADE of timestamp 1015 1037 35 9.8 101.5 0.2X -YEAR of timestamp 1011 1013 4 9.9 101.1 0.2X -ISOYEAR of timestamp 1121 1126 7 8.9 112.1 0.2X -QUARTER of timestamp 1243 1247 4 8.0 124.3 0.2X -MONTH of timestamp 1000 1009 14 10.0 100.0 0.2X -WEEK of timestamp 1481 1492 11 6.8 148.1 0.2X -DAY of timestamp 999 1000 1 10.0 99.9 0.2X -DAYOFWEEK of timestamp 1114 1132 20 9.0 111.4 0.2X -DOW of timestamp 1135 1147 17 8.8 113.5 0.2X -ISODOW of timestamp 1070 1079 10 9.3 107.0 0.2X -DOY of timestamp 1018 1027 8 9.8 101.8 0.2X -HOUR of timestamp 413 419 9 24.2 41.3 0.6X -MINUTE of timestamp 411 414 3 24.3 41.1 0.6X -SECOND of timestamp 410 413 3 24.4 41.0 0.6X -MILLISECONDS of timestamp 29225 29264 52 0.3 2922.5 0.0X -MICROSECONDS of timestamp 507 512 7 19.7 50.7 0.4X -EPOCH of timestamp 23565 23608 56 0.4 2356.5 0.0X +cast to timestamp 327 333 10 30.6 32.7 1.0X +MILLENNIUM of timestamp 1292 1296 4 7.7 129.2 0.3X +CENTURY of timestamp 1301 1305 6 7.7 130.1 0.3X +DECADE of timestamp 1200 1204 6 8.3 120.0 0.3X +YEAR of timestamp 1185 1193 8 8.4 118.5 0.3X +ISOYEAR of timestamp 1449 1469 18 6.9 144.9 0.2X +QUARTER of timestamp 1497 1505 7 6.7 149.7 0.2X +MONTH of timestamp 1185 1188 3 8.4 118.5 0.3X +WEEK of timestamp 1901 1909 7 5.3 190.1 0.2X +DAY of timestamp 1278 1282 4 7.8 127.8 0.3X +DAYOFWEEK of timestamp 1371 1376 5 7.3 137.1 0.2X +DOW of timestamp 1361 1372 17 7.3 136.1 0.2X +ISODOW of timestamp 1299 1306 9 7.7 129.9 0.3X +DOY of timestamp 1216 1219 4 8.2 121.6 0.3X +HOUR of timestamp 352 356 5 28.4 35.2 0.9X +MINUTE of timestamp 350 369 17 28.6 35.0 0.9X +SECOND of timestamp 351 364 19 28.5 35.1 0.9X +MILLISECONDS of timestamp 36989 37022 52 0.3 3698.9 0.0X +MICROSECONDS of timestamp 473 476 2 21.1 47.3 0.7X +EPOCH of timestamp 29890 29908 27 0.3 2989.0 0.0X -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 848 852 3 11.8 84.8 1.0X -MILLENNIUM of date 1083 1086 3 9.2 108.3 0.8X -CENTURY of date 1081 1084 5 9.3 108.1 0.8X -DECADE of date 1010 1014 4 9.9 101.0 0.8X -YEAR of date 992 1000 9 10.1 99.2 0.9X -ISOYEAR of date 1111 1116 6 9.0 111.1 0.8X -QUARTER of date 1230 1249 17 8.1 123.0 0.7X -MONTH of date 996 1008 11 10.0 99.6 0.9X -WEEK of date 1487 1516 28 6.7 148.7 0.6X -DAY of date 993 999 6 10.1 99.3 0.9X -DAYOFWEEK of date 1113 1118 5 9.0 111.3 0.8X -DOW of date 1113 1116 2 9.0 111.3 0.8X -ISODOW of date 1069 1072 3 9.4 106.9 0.8X -DOY of date 1027 1028 1 9.7 102.7 0.8X -HOUR of date 1707 1710 3 5.9 170.7 0.5X -MINUTE of date 1704 1705 2 5.9 170.4 0.5X -SECOND of date 1701 1705 4 5.9 170.1 0.5X -MILLISECONDS of date 2229 2238 9 4.5 222.9 0.4X -MICROSECONDS of date 1801 1808 12 5.6 180.1 0.5X -EPOCH of date 24783 24817 31 0.4 2478.3 0.0X +cast to date 1005 1006 1 9.9 100.5 1.0X +MILLENNIUM of date 1295 1300 5 7.7 129.5 0.8X +CENTURY of date 1297 1298 1 7.7 129.7 0.8X +DECADE of date 1198 1208 13 8.3 119.8 0.8X +YEAR of date 1184 1193 13 8.4 118.4 0.8X +ISOYEAR of date 1445 1460 13 6.9 144.5 0.7X +QUARTER of date 1495 1500 4 6.7 149.5 0.7X +MONTH of date 1176 1179 3 8.5 117.6 0.9X +WEEK of date 1893 1904 15 5.3 189.3 0.5X +DAY of date 1275 1283 8 7.8 127.5 0.8X +DAYOFWEEK of date 1369 1373 4 7.3 136.9 0.7X +DOW of date 1353 1354 2 7.4 135.3 0.7X +ISODOW of date 1290 1290 1 7.8 129.0 0.8X +DOY of date 1208 1212 4 8.3 120.8 0.8X +HOUR of date 1446 1449 2 6.9 144.6 0.7X +MINUTE of date 1441 1442 1 6.9 144.1 0.7X +SECOND of date 1443 1450 8 6.9 144.3 0.7X +MILLISECONDS of date 2087 2089 3 4.8 208.7 0.5X +MICROSECONDS of date 1557 1570 21 6.4 155.7 0.6X +EPOCH of date 30980 31001 32 0.3 3098.0 0.0X diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt index b06b5c092b61a..5fa461fecd4d2 100644 --- a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt +++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt @@ -2,12 +2,11 @@ Write primitive arrays in dataset ================================================================================================ -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 - -Write an array in Dataset: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Int 437 / 529 19.2 52.1 1.0X -Double 638 / 670 13.1 76.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 +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 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala index bd2470ee20660..df122977fe5f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala @@ -22,11 +22,10 @@ import scala.collection.JavaConverters._ import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.execution.datasources.parquet.{SpecificParquetRecordReaderBase, VectorizedParquetRecordReader} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -44,21 +43,26 @@ import org.apache.spark.sql.vectorized.ColumnVector * Results will be written to "benchmarks/DataSourceReadBenchmark-results.txt". * }}} */ -object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { - val conf = new SparkConf() - .setAppName("DataSourceReadBenchmark") - // Since `spark.master` always exists, overrides this value - .set("spark.master", "local[1]") - .setIfMissing("spark.driver.memory", "3g") - .setIfMissing("spark.executor.memory", "3g") - .setIfMissing(UI_ENABLED, false) - - val spark = SparkSession.builder.config(conf).getOrCreate() - - // Set default configs. Individual cases will change them if necessary. - spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") +object DataSourceReadBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName("DataSourceReadBenchmark") + // Since `spark.master` always exists, overrides this value + .set("spark.master", "local[1]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + .setIfMissing(UI_ENABLED, false) + + val sparkSession = SparkSession.builder.config(conf).getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + + sparkSession + } def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) 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 b9086aa5cb378..a109b11b2d6d3 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 @@ -19,9 +19,7 @@ package org.apache.spark.sql.execution.benchmark import java.time.Instant -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.internal.SQLConf /** @@ -36,11 +34,7 @@ import org.apache.spark.sql.internal.SQLConf * Results will be written to "benchmarks/ExtractBenchmark-results.txt". * }}} */ -object ExtractBenchmark extends BenchmarkBase with SQLHelper { - private val spark: SparkSession = SparkSession.builder() - .master("local[1]") - .appName(this.getClass.getCanonicalName) - .getOrCreate() +object ExtractBenchmark extends SqlBasedBenchmark { private def doBenchmark(cardinality: Long, exprs: String*): Unit = { val sinceSecond = Instant.parse("2010-01-01T00:00:00Z").getEpochSecond diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index b040243717137..50ba50176c7f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -22,10 +22,9 @@ import java.io.File import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.functions.monotonically_increasing_id import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -41,17 +40,21 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType * Results will be written to "benchmarks/FilterPushdownBenchmark-results.txt". * }}} */ -object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { - - private val conf = new SparkConf() - .setAppName(this.getClass.getSimpleName) - // Since `spark.master` always exists, overrides this value - .set("spark.master", "local[1]") - .setIfMissing("spark.driver.memory", "3g") - .setIfMissing("spark.executor.memory", "3g") - .setIfMissing(UI_ENABLED, false) - .setIfMissing("orc.compression", "snappy") - .setIfMissing("spark.sql.parquet.compression.codec", "snappy") +object FilterPushdownBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName(this.getClass.getSimpleName) + // Since `spark.master` always exists, overrides this value + .set("spark.master", "local[1]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + .setIfMissing(UI_ENABLED, false) + .setIfMissing("orc.compression", "snappy") + .setIfMissing("spark.sql.parquet.compression.codec", "snappy") + + SparkSession.builder().config(conf).getOrCreate() + } private val numRows = 1024 * 1024 * 15 private val width = 5 @@ -59,8 +62,6 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { // For Parquet/ORC, we will use the same value for block size and compression size private val blockSize = org.apache.parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE - private val spark = SparkSession.builder().config(conf).getOrCreate() - def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala index 8b1c422e63a3f..e07921bf3aa74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.benchmark -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.SparkSession /** @@ -28,13 +28,16 @@ import org.apache.spark.sql.SparkSession * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/PrimitiveArrayBenchmark-results.txt". */ -object PrimitiveArrayBenchmark extends BenchmarkBase { - lazy val sparkSession = SparkSession.builder - .master("local[1]") - .appName("microbenchmark") - .config("spark.sql.shuffle.partitions", 1) - .config("spark.sql.autoBroadcastJoinThreshold", 1) - .getOrCreate() +object PrimitiveArrayBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() + } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Write primitive arrays in dataset") { @@ -43,11 +46,11 @@ object PrimitiveArrayBenchmark extends BenchmarkBase { } def writeDatasetArray(iters: Int): Unit = { - import sparkSession.implicits._ + import spark.implicits._ val count = 1024 * 1024 * 2 - val sc = sparkSession.sparkContext + val sc = spark.sparkContext val primitiveIntArray = Array.fill[Int](count)(65535) val dsInt = sc.parallelize(Seq(primitiveIntArray), 1).toDS dsInt.count // force to build dataset diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt index f3044da972497..0c394a340333a 100644 --- a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt @@ -2,44 +2,44 @@ Hive UDAF vs Spark AF ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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 -hive udaf vs spark af: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -hive udaf w/o group by 6370 / 6400 0.0 97193.6 1.0X -spark af w/o group by 54 / 63 1.2 820.8 118.4X -hive udaf w/ group by 4492 / 4507 0.0 68539.5 1.4X -spark af w/ group by w/o fallback 58 / 64 1.1 881.7 110.2X -spark af w/ group by w/ fallback 136 / 142 0.5 2075.0 46.8X +hive udaf vs spark af: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +hive udaf w/o group by 6741 6759 22 0.0 102864.5 1.0X +spark af w/o group by 56 66 9 1.2 851.6 120.8X +hive udaf w/ group by 4610 4642 25 0.0 70350.3 1.5X +spark af w/ group by w/o fallback 60 67 8 1.1 916.7 112.2X +spark af w/ group by w/ fallback 135 144 9 0.5 2065.6 49.8X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - typed_count ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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 -object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sort agg w/ group by 41500 / 41630 2.5 395.8 1.0X -object agg w/ group by w/o fallback 10075 / 10122 10.4 96.1 4.1X -object agg w/ group by w/ fallback 28131 / 28205 3.7 268.3 1.5X -sort agg w/o group by 6182 / 6221 17.0 59.0 6.7X -object agg w/o group by w/o fallback 5435 / 5468 19.3 51.8 7.6X +object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort agg w/ group by 41568 41894 461 2.5 396.4 1.0X +object agg w/ group by w/o fallback 10314 10494 149 10.2 98.4 4.0X +object agg w/ group by w/ fallback 26720 26951 326 3.9 254.8 1.6X +sort agg w/o group by 6638 6681 38 15.8 63.3 6.3X +object agg w/o group by w/o fallback 5665 5706 30 18.5 54.0 7.3X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - percentile_approx ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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 -object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sort agg w/ group by 970 / 1025 2.2 462.5 1.0X -object agg w/ group by w/o fallback 772 / 798 2.7 368.1 1.3X -object agg w/ group by w/ fallback 1013 / 1044 2.1 483.1 1.0X -sort agg w/o group by 751 / 781 2.8 358.0 1.3X -object agg w/o group by w/o fallback 772 / 814 2.7 368.0 1.3X +object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort agg w/ group by 794 862 33 2.6 378.8 1.0X +object agg w/ group by w/o fallback 605 622 10 3.5 288.5 1.3X +object agg w/ group by w/ fallback 840 860 15 2.5 400.5 0.9X +sort agg w/o group by 555 570 12 3.8 264.6 1.4X +object agg w/o group by w/o fallback 544 562 12 3.9 259.6 1.5X diff --git a/sql/hive/benchmarks/OrcReadBenchmark-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-results.txt index caa78b9a8f102..c47cf27bf617a 100644 --- a/sql/hive/benchmarks/OrcReadBenchmark-results.txt +++ b/sql/hive/benchmarks/OrcReadBenchmark-results.txt @@ -2,155 +2,155 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1725 / 1759 9.1 109.7 1.0X -Native ORC Vectorized 272 / 316 57.8 17.3 6.3X -Hive built-in ORC 1970 / 1987 8.0 125.3 0.9X +SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1843 1958 162 8.5 117.2 1.0X +Native ORC Vectorized 321 355 31 48.9 20.4 5.7X +Hive built-in ORC 2143 2175 44 7.3 136.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 -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1633 / 1672 9.6 103.8 1.0X -Native ORC Vectorized 238 / 255 66.0 15.1 6.9X -Hive built-in ORC 2293 / 2305 6.9 145.8 0.7X +SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1987 2020 47 7.9 126.3 1.0X +Native ORC Vectorized 276 299 25 57.0 17.6 7.2X +Hive built-in ORC 2350 2357 10 6.7 149.4 0.8X -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 -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1677 / 1699 9.4 106.6 1.0X -Native ORC Vectorized 325 / 342 48.3 20.7 5.2X -Hive built-in ORC 2561 / 2569 6.1 162.8 0.7X +SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2092 2115 32 7.5 133.0 1.0X +Native ORC Vectorized 360 373 18 43.6 22.9 5.8X +Hive built-in ORC 2550 2557 9 6.2 162.2 0.8X -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 -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1791 / 1795 8.8 113.9 1.0X -Native ORC Vectorized 400 / 408 39.3 25.4 4.5X -Hive built-in ORC 2713 / 2720 5.8 172.5 0.7X +SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2173 2188 21 7.2 138.2 1.0X +Native ORC Vectorized 435 448 14 36.2 27.7 5.0X +Hive built-in ORC 2683 2690 10 5.9 170.6 0.8X -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 -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1791 / 1805 8.8 113.8 1.0X -Native ORC Vectorized 433 / 438 36.3 27.5 4.1X -Hive built-in ORC 2690 / 2803 5.8 171.0 0.7X +SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2233 2323 127 7.0 142.0 1.0X +Native ORC Vectorized 475 483 13 33.1 30.2 4.7X +Hive built-in ORC 2605 2610 6 6.0 165.7 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 -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1911 / 1930 8.2 121.5 1.0X -Native ORC Vectorized 543 / 552 29.0 34.5 3.5X -Hive built-in ORC 2967 / 3065 5.3 188.6 0.6X +SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2367 2384 24 6.6 150.5 1.0X +Native ORC Vectorized 600 641 69 26.2 38.1 3.9X +Hive built-in ORC 2860 2877 24 5.5 181.9 0.8X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 4160 / 4188 2.5 396.7 1.0X -Native ORC Vectorized 2405 / 2406 4.4 229.4 1.7X -Hive built-in ORC 5514 / 5562 1.9 525.9 0.8X +Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 4253 4330 108 2.5 405.6 1.0X +Native ORC Vectorized 2295 2301 8 4.6 218.9 1.9X +Hive built-in ORC 5364 5465 144 2.0 511.5 0.8X ================================================================================================ Partitioned Table Scan ================================================================================================ -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 -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column - Native ORC MR 1863 / 1867 8.4 118.4 1.0X -Data column - Native ORC Vectorized 411 / 418 38.2 26.2 4.5X -Data column - Hive built-in ORC 3297 / 3308 4.8 209.6 0.6X -Partition column - Native ORC MR 1505 / 1506 10.4 95.7 1.2X -Partition column - Native ORC Vectorized 80 / 93 195.6 5.1 23.2X -Partition column - Hive built-in ORC 1960 / 1979 8.0 124.6 1.0X -Both columns - Native ORC MR 2076 / 2090 7.6 132.0 0.9X -Both columns - Native ORC Vectorized 450 / 463 34.9 28.6 4.1X -Both columns - Hive built-in ORC 3528 / 3548 4.5 224.3 0.5X +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column - Native ORC MR 2443 2448 6 6.4 155.3 1.0X +Data column - Native ORC Vectorized 446 473 44 35.3 28.3 5.5X +Data column - Hive built-in ORC 2868 2877 12 5.5 182.4 0.9X +Partition column - Native ORC MR 1623 1656 47 9.7 103.2 1.5X +Partition column - Native ORC Vectorized 112 121 14 140.8 7.1 21.9X +Partition column - Hive built-in ORC 1846 1850 5 8.5 117.4 1.3X +Both columns - Native ORC MR 2610 2635 36 6.0 165.9 0.9X +Both columns - Native ORC Vectorized 492 508 19 32.0 31.3 5.0X +Both columns - Hive built-in ORC 2969 2973 4 5.3 188.8 0.8X ================================================================================================ Repeated String Scan ================================================================================================ -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 -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1727 / 1733 6.1 164.7 1.0X -Native ORC Vectorized 375 / 379 28.0 35.7 4.6X -Hive built-in ORC 2665 / 2666 3.9 254.2 0.6X +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2056 2064 11 5.1 196.1 1.0X +Native ORC Vectorized 415 421 7 25.3 39.6 5.0X +Hive built-in ORC 2710 2722 17 3.9 258.4 0.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 3324 / 3325 3.2 317.0 1.0X -Native ORC Vectorized 1085 / 1106 9.7 103.4 3.1X -Hive built-in ORC 5272 / 5299 2.0 502.8 0.6X +String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 3655 3674 27 2.9 348.6 1.0X +Native ORC Vectorized 1166 1167 1 9.0 111.2 3.1X +Hive built-in ORC 5268 5305 52 2.0 502.4 0.7X -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 -String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 3045 / 3046 3.4 290.4 1.0X -Native ORC Vectorized 1248 / 1260 8.4 119.0 2.4X -Hive built-in ORC 3989 / 3999 2.6 380.4 0.8X +String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 3447 3467 27 3.0 328.8 1.0X +Native ORC Vectorized 1222 1223 1 8.6 116.6 2.8X +Hive built-in ORC 3947 3959 18 2.7 376.4 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 -String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1692 / 1694 6.2 161.3 1.0X -Native ORC Vectorized 471 / 493 22.3 44.9 3.6X -Hive built-in ORC 2398 / 2411 4.4 228.7 0.7X +String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1912 1917 6 5.5 182.4 1.0X +Native ORC Vectorized 477 484 5 22.0 45.5 4.0X +Hive built-in ORC 2374 2386 17 4.4 226.4 0.8X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +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/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1371 / 1379 0.8 1307.5 1.0X -Native ORC Vectorized 121 / 135 8.6 115.8 11.3X -Hive built-in ORC 521 / 561 2.0 497.1 2.6X +Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 290 350 102 3.6 276.1 1.0X +Native ORC Vectorized 155 166 15 6.7 148.2 1.9X +Hive built-in ORC 520 531 8 2.0 495.8 0.6X -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 -Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 2711 / 2767 0.4 2585.5 1.0X -Native ORC Vectorized 210 / 232 5.0 200.5 12.9X -Hive built-in ORC 764 / 775 1.4 728.3 3.5X +Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 365 406 73 2.9 347.9 1.0X +Native ORC Vectorized 232 246 20 4.5 221.6 1.6X +Hive built-in ORC 794 864 62 1.3 757.6 0.5X -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 -Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 3979 / 3988 0.3 3794.4 1.0X -Native ORC Vectorized 357 / 366 2.9 340.2 11.2X -Hive built-in ORC 1091 / 1095 1.0 1040.5 3.6X +Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 501 544 40 2.1 477.6 1.0X +Native ORC Vectorized 365 386 33 2.9 348.0 1.4X +Hive built-in ORC 1153 1153 0 0.9 1099.8 0.4X diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index 3226e3a5f318a..c475c7b21ab95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -21,11 +21,10 @@ import scala.concurrent.duration._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.hive.execution.TestingTypedCount import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf @@ -44,9 +43,10 @@ import org.apache.spark.sql.types.LongType * Results will be written to "benchmarks/ObjectHashAggregateExecBenchmark-results.txt". * }}} */ -object ObjectHashAggregateExecBenchmark extends BenchmarkBase with SQLHelper { +object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = TestHive.sparkSession - private val spark: SparkSession = TestHive.sparkSession private val sql = spark.sql _ import spark.implicits._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index c03ae144a1595..f28b9be60d3cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -22,9 +22,9 @@ import java.io.File import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -43,18 +43,23 @@ import org.apache.spark.sql.types._ * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources. */ // scalastyle:off line.size.limit -object OrcReadBenchmark extends BenchmarkBase with SQLHelper { - val conf = new SparkConf() - conf.set("orc.compression", "snappy") - - private val spark = SparkSession.builder() - .master("local[1]") - .appName("OrcReadBenchmark") - .config(conf) - .getOrCreate() - - // Set default configs. Individual cases will change them if necessary. - spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") +object OrcReadBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + conf.set("orc.compression", "snappy") + + val sparkSession = SparkSession.builder() + .master("local[1]") + .appName("OrcReadBenchmark") + .config(conf) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") + + sparkSession + } def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) From b917a6593dc969b9b766259eb8cbbd6e90e0dc53 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 18 Sep 2019 22:30:28 -0700 Subject: [PATCH 48/87] [SPARK-28989][SQL] Add a SQLConf `spark.sql.ansi.enabled` ### What changes were proposed in this pull request? Currently, there are new configurations for compatibility with ANSI SQL: * `spark.sql.parser.ansi.enabled` * `spark.sql.decimalOperations.nullOnOverflow` * `spark.sql.failOnIntegralTypeOverflow` This PR is to add new configuration `spark.sql.ansi.enabled` and remove the 3 options above. When the configuration is true, Spark tries to conform to the ANSI SQL specification. It will be disabled by default. ### Why are the changes needed? Make it simple and straightforward. ### Does this PR introduce any user-facing change? The new features for ANSI compatibility will be set via one configuration `spark.sql.ansi.enabled`. ### How was this patch tested? Existing unit tests. Closes #25693 from gengliangwang/ansiEnabled. Lead-authored-by: Gengliang Wang Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/sql-keywords.md | 8 ++-- .../sql/catalyst/CatalystTypeConverters.scala | 2 +- .../sql/catalyst/SerializerBuildHelper.scala | 2 +- .../catalyst/analysis/DecimalPrecision.scala | 2 +- .../sql/catalyst/encoders/RowEncoder.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 8 ++-- .../catalyst/expressions/aggregate/Sum.scala | 2 +- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../expressions/decimalExpressions.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql/catalyst/parser/ParseDriver.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 41 +++++-------------- .../encoders/ExpressionEncoderSuite.scala | 8 ++-- .../catalyst/encoders/RowEncoderSuite.scala | 4 +- .../ArithmeticExpressionSuite.scala | 24 +++++------ .../sql/catalyst/expressions/CastSuite.scala | 12 +++--- .../expressions/DecimalExpressionSuite.scala | 4 +- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +- .../parser/ExpressionParserSuite.scala | 10 ++--- .../parser/TableIdentifierParserSuite.scala | 2 +- .../sql-tests/inputs/ansi/interval.sql | 4 +- .../inputs/decimalArithmeticOperations.sql | 2 +- .../resources/sql-tests/inputs/pgSQL/text.sql | 6 +-- .../sql-tests/results/ansi/interval.sql.out | 8 ++-- .../decimalArithmeticOperations.sql.out | 4 +- .../sql-tests/results/pgSQL/text.sql.out | 8 ++-- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +-- .../apache/spark/sql/SQLQueryTestSuite.scala | 8 ++-- .../ThriftServerQueryTestSuite.scala | 2 +- 29 files changed, 86 insertions(+), 109 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 16ce35d55ed7b..c55e589aa8c3f 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.parser.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.parser.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.parser.ansi.enabled=true`. +When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: 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. -By default `spark.sql.parser.ansi.enabled` is false. +By default `spark.sql.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 488252aa0c7b5..34d2f45e715e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -344,7 +344,7 @@ object CatalystTypeConverters { private class DecimalConverter(dataType: DecimalType) extends CatalystTypeConverter[Any, JavaBigDecimal, Decimal] { - private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + private val nullOnOverflow = !SQLConf.get.ansiEnabled override def toCatalystImpl(scalaValue: Any): Decimal = { val decimal = scalaValue match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index 75c278e781140..026ff6f2983fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -26,7 +26,7 @@ import org.apache.spark.unsafe.types.UTF8String object SerializerBuildHelper { - private def nullOnOverflow: Boolean = SQLConf.get.decimalOperationsNullOnOverflow + private def nullOnOverflow: Boolean = !SQLConf.get.ansiEnabled def createSerializerForBoolean(inputObject: Expression): Expression = { Invoke(inputObject, "booleanValue", BooleanType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index 856c2ed828002..a64befecb68d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -82,7 +82,7 @@ object DecimalPrecision extends TypeCoercionRule { PromotePrecision(Cast(e, dataType)) } - private def nullOnOverflow: Boolean = SQLConf.get.decimalOperationsNullOnOverflow + private def nullOnOverflow: Boolean = !SQLConf.get.ansiEnabled override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan resolveOperators { // fix decimal precision for expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index afe8a23f8f150..765018f07d87a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -114,7 +114,7 @@ object RowEncoder { d, "fromDecimal", inputObject :: Nil, - returnNullable = false), d, SQLConf.get.decimalOperationsNullOnOverflow) + returnNullable = false), d, !SQLConf.get.ansiEnabled) case StringType => createSerializerForString(inputObject) 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 d1943f02f85e5..5001b4350d778 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 @@ -289,7 +289,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private lazy val dateFormatter = DateFormatter() private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) - private val failOnIntegralTypeOverflow = SQLConf.get.failOnIntegralTypeOverflow + private val failOnIntegralTypeOverflow = SQLConf.get.ansiEnabled // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { @@ -600,13 +600,13 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } - private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + 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 * either returns null or throws an exception according to the value set for - * `spark.sql.decimalOperations.nullOnOverflow`. + * `spark.sql.ansi.enabled`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ @@ -625,7 +625,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String /** * Create new `Decimal` with precision and scale given in `decimalType` (if any). - * If overflow occurs, if `spark.sql.decimalOperations.nullOnOverflow` is true, null is returned; + * If overflow occurs, if `spark.sql.ansi.enabled` is false, null is returned; * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index d04fe9249d064..c2ab8adfaef67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -91,7 +91,7 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast } override lazy val evaluateExpression: Expression = resultType match { - case d: DecimalType => CheckOverflow(sum, d, SQLConf.get.decimalOperationsNullOnOverflow) + case d: DecimalType => CheckOverflow(sum, d, !SQLConf.get.ansiEnabled) case _ => sum } 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 2ed82d99fe2bb..e4276e33acbd2 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 @@ -35,7 +35,7 @@ import org.apache.spark.unsafe.types.CalendarInterval """) case class UnaryMinus(child: Expression) extends UnaryExpression with ExpectsInputTypes with NullIntolerant { - private val checkOverflow = SQLConf.get.failOnIntegralTypeOverflow + private val checkOverflow = SQLConf.get.ansiEnabled override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection.NumericAndInterval) @@ -136,7 +136,7 @@ case class Abs(child: Expression) abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { - protected val checkOverflow = SQLConf.get.failOnIntegralTypeOverflow + protected val checkOverflow = SQLConf.get.ansiEnabled override def dataType: DataType = left.dataType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index b5b712cda8ea3..7b2489e682661 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -47,7 +47,7 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { */ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends UnaryExpression { - private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + private val nullOnOverflow = !SQLConf.get.ansiEnabled override def dataType: DataType = DecimalType(precision, scale) override def nullable: Boolean = child.nullable || nullOnOverflow 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 9335be5b239b6..c334d09b691ea 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 @@ -1363,7 +1363,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } override def visitCurrentDatetime(ctx: CurrentDatetimeContext): Expression = withOrigin(ctx) { - if (conf.ansiParserEnabled) { + if (conf.ansiEnabled) { ctx.name.getType match { case SqlBaseParser.CURRENT_DATE => CurrentDate() 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 932e795f1d0bc..85998e33140d0 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 @@ -92,7 +92,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - lexer.ansi = SQLConf.get.ansiParserEnabled + lexer.ansi = SQLConf.get.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -100,7 +100,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - parser.ansi = SQLConf.get.ansiParserEnabled + parser.ansi = SQLConf.get.ansiEnabled try { try { 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 4f3e39ad49afe..0ec661fc16c88 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 @@ -411,12 +411,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ANSI_SQL_PARSER = - buildConf("spark.sql.parser.ansi.enabled") - .doc("When true, tries to conform to ANSI SQL syntax.") - .booleanConf - .createWithDefault(false) - val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + @@ -1557,16 +1551,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val DECIMAL_OPERATIONS_NULL_ON_OVERFLOW = - buildConf("spark.sql.decimalOperations.nullOnOverflow") - .internal() - .doc("When true (default), if an overflow on a decimal occurs, then NULL is returned. " + - "Spark's older versions and Hive behave in this way. If turned to false, SQL ANSI 2011 " + - "specification will be followed instead: an arithmetic exception is thrown, as most " + - "of the SQL databases do.") - .booleanConf - .createWithDefault(true) - val LITERAL_PICK_MINIMUM_PRECISION = buildConf("spark.sql.legacy.literal.pickMinimumPrecision") .internal() @@ -1723,6 +1707,14 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createOptional + val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + + "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + + "the SQL parser.") + .booleanConf + .createWithDefault(false) + val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -1886,15 +1878,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val FAIL_ON_INTEGRAL_TYPE_OVERFLOW = - buildConf("spark.sql.failOnIntegralTypeOverflow") - .doc("If it is set to true, all operations on integral fields throw an " + - "exception if an overflow occurs. If it is false (default), in case of overflow a wrong " + - "result is returned.") - .internal() - .booleanConf - .createWithDefault(false) - val LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE = buildConf("spark.sql.legacy.parser.havingWithoutGroupByAsWhere") .internal() @@ -2195,8 +2178,6 @@ class SQLConf extends Serializable with Logging { def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED) - def ansiParserEnabled: Boolean = getConf(ANSI_SQL_PARSER) - def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS) def fileCompressionFactor: Double = getConf(FILE_COMPRESSION_FACTOR) @@ -2418,10 +2399,6 @@ class SQLConf extends Serializable with Logging { def decimalOperationsAllowPrecisionLoss: Boolean = getConf(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS) - def decimalOperationsNullOnOverflow: Boolean = getConf(DECIMAL_OPERATIONS_NULL_ON_OVERFLOW) - - def failOnIntegralTypeOverflow: Boolean = getConf(FAIL_ON_INTEGRAL_TYPE_OVERFLOW) - def literalPickMinimumPrecision: Boolean = getConf(LITERAL_PICK_MINIMUM_PRECISION) def continuousStreamingEpochBacklogQueueSize: Int = @@ -2454,6 +2431,8 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: Option[StoreAssignmentPolicy.Value] = getConf(STORE_ASSIGNMENT_POLICY).map(StoreAssignmentPolicy.withName) + def ansiEnabled: Boolean = getConf(ANSI_ENABLED) + def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) def serializerNestedSchemaPruningEnabled: Boolean = 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 9380c7e3f5f72..d2acfb9686700 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 @@ -432,16 +432,16 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes } private def testOverflowingBigNumeric[T: TypeTag](bigNumeric: T, testName: String): Unit = { - Seq(true, false).foreach { allowNullOnOverflow => + Seq(true, false).foreach { ansiEnabled => testAndVerifyNotLeakingReflectionObjects( - s"overflowing $testName, allowNullOnOverflow=$allowNullOnOverflow") { + s"overflowing $testName, ansiEnabled=$ansiEnabled") { withSQLConf( - SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> allowNullOnOverflow.toString + SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString ) { // Need to construct Encoder here rather than implicitly resolving it // so that SQLConf changes are respected. val encoder = ExpressionEncoder[T]() - if (allowNullOnOverflow) { + if (!ansiEnabled) { val convertedBack = encoder.resolveAndBind().fromRow(encoder.toRow(bigNumeric)) assert(convertedBack === null) } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 5d21e4a2a83ca..1a1cab823d4f3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -169,7 +169,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } private def testDecimalOverflow(schema: StructType, row: Row): Unit = { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val encoder = RowEncoder(schema).resolveAndBind() intercept[Exception] { encoder.toRow(row) @@ -182,7 +182,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 729e2f529b97f..ad8b1a1673679 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -61,7 +61,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Add(positiveLongLit, negativeLongLit), -1L) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Add, tpe, tpe) } @@ -80,7 +80,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(UnaryMinus(Literal(Int.MinValue)), Int.MinValue) checkEvaluation(UnaryMinus(Literal(Short.MinValue)), Short.MinValue) checkEvaluation(UnaryMinus(Literal(Byte.MinValue)), Byte.MinValue) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( UnaryMinus(Literal(Long.MinValue)), "overflow") checkExceptionInExpression[ArithmeticException]( @@ -122,7 +122,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Subtract(positiveLongLit, negativeLongLit), positiveLong - negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Subtract, tpe, tpe) } @@ -144,7 +144,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Multiply(positiveLongLit, negativeLongLit), positiveLong * negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Multiply, tpe, tpe) } @@ -445,12 +445,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minLongLiteral, minLongLiteral) val e5 = Subtract(minLongLiteral, maxLongLiteral) val e6 = Multiply(minLongLiteral, minLongLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Long.MinValue) checkEvaluation(e2, Long.MinValue) checkEvaluation(e3, -2L) @@ -469,12 +469,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minIntLiteral, minIntLiteral) val e5 = Subtract(minIntLiteral, maxIntLiteral) val e6 = Multiply(minIntLiteral, minIntLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Int.MinValue) checkEvaluation(e2, Int.MinValue) checkEvaluation(e3, -2) @@ -493,12 +493,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minShortLiteral, minShortLiteral) val e5 = Subtract(minShortLiteral, maxShortLiteral) val e6 = Multiply(minShortLiteral, minShortLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Short.MinValue) checkEvaluation(e2, Short.MinValue) checkEvaluation(e3, (-2).toShort) @@ -517,12 +517,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minByteLiteral, minByteLiteral) val e5 = Subtract(minByteLiteral, maxByteLiteral) val e6 = Multiply(minByteLiteral, minByteLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Byte.MinValue) checkEvaluation(e2, Byte.MinValue) checkEvaluation(e3, (-2).toByte) 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 7a781295a7957..ffb14e2838687 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 @@ -1044,14 +1044,14 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + 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.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( Cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( @@ -1117,7 +1117,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to byte with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -1142,7 +1142,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to short with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1167,7 +1167,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to int with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1184,7 +1184,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to long with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index fc5e8dc5ee7f1..36bc3db580400 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -32,7 +32,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) @@ -41,7 +41,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen(overflowExpr, null) checkEvaluationWithUnsafeProjection(overflowExpr, null) } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index 981ef57c051fd..c5ffc381b58e2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -57,7 +57,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, @@ -69,7 +69,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(e2.getCause.isInstanceOf[ArithmeticException]) } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, 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 ee89980d3eedd..5da2bf059758d 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 @@ -605,7 +605,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(s"interval $intervalValue", expected) // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } } @@ -682,12 +682,12 @@ class ExpressionParserSuite extends AnalysisTest { test("SPARK-23264 Interval Compatibility tests") { def checkIntervals(intervalValue: String, expected: Literal): Unit = { - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val aliases = defaultParser.parseExpression(intervalValue).collect { case a @ Alias(_: Literal, name) if intervalUnits.exists { unit => name.startsWith(unit) } => a @@ -785,12 +785,12 @@ class ExpressionParserSuite extends AnalysisTest { } test("current date/timestamp braceless expressions") { - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 81f0bd024a80e..a9216174804d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -658,7 +658,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { } test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { reservedKeywordsInAnsiMode.foreach { keyword => val errMsg = intercept[ParseException] { parseTableIdentifier(keyword) diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql index f2f4b02c8634b..4a831855a091a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -1,5 +1,5 @@ -- Turns on ANSI mode -SET spark.sql.parser.ansi.enabled=true; +SET spark.sql.ansi.enabled=true; select '1' second, @@ -185,4 +185,4 @@ select date '2012-01-01' + interval (a + 1) day from t; select date '2012-01-01' + (a + 1) day from t; -- Turns off ANSI mode -SET spark.sql.parser.ansi.enabled=false; +SET spark.sql.ansi.enabled=false; diff --git a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql index 35f2be46cd130..a946123f709c7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql @@ -84,7 +84,7 @@ select 123456789123456789.1234567890 * 1.123456789123456789; select 12345678912345.123456789123 / 0.000000012345678; -- throw an exception instead of returning NULL, according to SQL ANSI 2011 -set spark.sql.decimalOperations.nullOnOverflow=false; +set spark.sql.ansi.enabled=true; -- test operations between decimals and constants select id, a*10, b/10 from decimals_test order by id; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql index 04d3acc145e95..7abf903bc6bee 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql @@ -45,10 +45,10 @@ select concat_ws('',10,20,null,30); select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); -- [SPARK-28036] Built-in udf left/right has inconsistent behavior --- [SPARK-28479] Parser error when enabling ANSI mode -set spark.sql.parser.ansi.enabled=false; +-- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode +set spark.sql.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.parser.ansi.enabled=true; +set spark.sql.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 13f72614f5778..43ad3c3f539f1 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -3,11 +3,11 @@ -- !query 0 -SET spark.sql.parser.ansi.enabled=true +SET spark.sql.ansi.enabled=true -- !query 0 schema struct -- !query 0 output -spark.sql.parser.ansi.enabled true +spark.sql.ansi.enabled true -- !query 1 @@ -432,8 +432,8 @@ select date '2012-01-01' + (a + 1) day from t -- !query 34 -SET spark.sql.parser.ansi.enabled=false +SET spark.sql.ansi.enabled=false -- !query 34 schema struct -- !query 34 output -spark.sql.parser.ansi.enabled false +spark.sql.ansi.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 217233bfad378..00e139d90f488 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -328,11 +328,11 @@ NULL -- !query 39 -set spark.sql.decimalOperations.nullOnOverflow=false +set spark.sql.ansi.enabled=true -- !query 39 schema struct -- !query 39 output -spark.sql.decimalOperations.nullOnOverflow false +spark.sql.ansi.enabled true -- !query 40 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out index 352b0232e8945..2e1d639974ec6 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out @@ -151,11 +151,11 @@ edcba -- !query 18 -set spark.sql.parser.ansi.enabled=false +set spark.sql.ansi.enabled=false -- !query 18 schema struct -- !query 18 output -spark.sql.parser.ansi.enabled false +spark.sql.ansi.enabled false -- !query 19 @@ -177,11 +177,11 @@ struct -- !query 20 -set spark.sql.parser.ansi.enabled=true +set spark.sql.ansi.enabled=true -- !query 20 schema struct -- !query 20 output -spark.sql.parser.ansi.enabled true +spark.sql.ansi.enabled true -- !query 21 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 b4ddfecaee469..6a2eac9e7e21c 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 @@ -161,10 +161,10 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { DecimalData(BigDecimal("1"* 20 + ".123"), BigDecimal("1"* 20 + ".123")) :: DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() - Seq(true, false).foreach { nullOnOverflow => - withSQLConf((SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key, nullOnOverflow.toString)) { + Seq(true, false).foreach { ansiEnabled => + withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { val structDf = largeDecimals.select("a").agg(sum("a")) - if (nullOnOverflow) { + if (!ansiEnabled) { checkAnswer(structDf, Row(null)) } else { val e = intercept[SparkException] { 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 cc92d6556387d..abecaa7a9b16a 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 @@ -308,13 +308,11 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("vol", (s: String) => s) // PostgreSQL enabled cartesian product by default. localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) - localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) - localSparkSession.conf.set(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, true) - // Propagate the SQL conf FAIL_ON_INTEGRAL_TYPE_OVERFLOW to executor. + // Propagate the SQL conf ANSI_ENABLED to executor. // TODO: remove this after SPARK-29122 is resolved. - localSparkSession.sparkContext.setLocalProperty( - SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, "true") + localSparkSession.sparkContext.setLocalProperty(SQLConf.ANSI_ENABLED.key, "true") case _ => } 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 1f7b3feae47b5..381b8f2324ca6 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 @@ -110,7 +110,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { case _: PgSQLTest => // PostgreSQL enabled cartesian product by default. statement.execute(s"SET ${SQLConf.CROSS_JOINS_ENABLED.key} = true") - statement.execute(s"SET ${SQLConf.ANSI_SQL_PARSER.key} = true") + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") case _ => } From 3bf43fb60d6f8aba23eaa1e43405024725b50f22 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 Sep 2019 00:24:15 -0700 Subject: [PATCH 49/87] [SPARK-29159][BUILD] Increase ReservedCodeCacheSize to 1G ### What changes were proposed in this pull request? This PR aims to increase the JVM CodeCacheSize from 0.5G to 1G. ### Why are the changes needed? After upgrading to `Scala 2.12.10`, the following is observed during building. ``` 2019-09-18T20:49:23.5030586Z OpenJDK 64-Bit Server VM warning: CodeCache is full. Compiler has been disabled. 2019-09-18T20:49:23.5032920Z OpenJDK 64-Bit Server VM warning: Try increasing the code cache size using -XX:ReservedCodeCacheSize= 2019-09-18T20:49:23.5034959Z CodeCache: size=524288Kb used=521399Kb max_used=521423Kb free=2888Kb 2019-09-18T20:49:23.5035472Z bounds [0x00007fa62c000000, 0x00007fa64c000000, 0x00007fa64c000000] 2019-09-18T20:49:23.5035781Z total_blobs=156549 nmethods=155863 adapters=592 2019-09-18T20:49:23.5036090Z compilation: disabled (not enough contiguous free space left) ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually check the Jenkins or GitHub Action build log (which should not have the above). Closes #25836 from dongjoon-hyun/SPARK-CODE-CACHE-1G. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 +- build/mvn | 2 +- dev/appveyor-install-dependencies.ps1 | 2 +- dev/make-distribution.sh | 2 +- docs/building-spark.md | 4 ++-- pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index dfed834c1d08e..99dd1d04a60d1 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -29,6 +29,6 @@ jobs: version: ${{ matrix.java }} - name: Build with Maven run: | - export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} package diff --git a/build/mvn b/build/mvn index f68377b3ddc71..3628be9880253 100755 --- a/build/mvn +++ b/build/mvn @@ -22,7 +22,7 @@ _DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" # Preserve the calling directory _CALLING_DIR="$(pwd)" # Options used during compilation -_COMPILE_JVM_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" +_COMPILE_JVM_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g" # Installs any application tarball given a URL, the expected tarball name, # and, optionally, a checkable binary path to determine if the binary has diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index d33a107cc86a5..e0976e66db29f 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -90,7 +90,7 @@ Invoke-Expression "7z.exe x maven.zip" # add maven to environment variables $env:PATH = "$tools\apache-maven-$mavenVer\bin;" + $env:PATH $env:M2_HOME = "$tools\apache-maven-$mavenVer" -$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=512m" +$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=1g" Pop-Location diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index a550af93feecd..cd18b6870e07c 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -160,7 +160,7 @@ fi # Build uber fat JAR cd "$SPARK_HOME" -export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=512m}" +export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=1g}" # Store the command as an array because $MVN variable might have spaces in it. # Normal quoting tricks don't work. diff --git a/docs/building-spark.md b/docs/building-spark.md index 37f898645da68..9fd8b20d2dde6 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -34,7 +34,7 @@ Spark requires Scala 2.12; support for Scala 2.11 was removed in Spark 3.0.0. You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`: - export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g" (The `ReservedCodeCacheSize` setting is optional but recommended.) If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following: @@ -160,7 +160,7 @@ prompt. Configure the JVM options for SBT in `.jvmopts` at the project root, for example: -Xmx2g - -XX:ReservedCodeCacheSize=512m + -XX:ReservedCodeCacheSize=1g For the meanings of these two options, please carefully read the [Setting up Maven's Memory Usage section](https://spark.apache.org/docs/latest/building-spark.html#setting-up-mavens-memory-usage). diff --git a/pom.xml b/pom.xml index a78a5beb814f5..2f29ad07f1d90 100644 --- a/pom.xml +++ b/pom.xml @@ -240,7 +240,7 @@ --> ${session.executionRootDirectory} - 512m + 1g diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index cd3e8b234c189..84d99115a93ac 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -141,7 +141,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx4g -XX:ReservedCodeCacheSize=512m ${extraScalaTestArgs} + -ea -Xmx4g -XX:ReservedCodeCacheSize=1g ${extraScalaTestArgs} file:src/test/resources/log4j.properties From e97b55d32285052a1f76cca35377c4b21eb2e7d7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 19 Sep 2019 08:17:25 -0500 Subject: [PATCH 50/87] [SPARK-28985][PYTHON][ML] Add common classes (JavaPredictor/JavaClassificationModel/JavaProbabilisticClassifier) in PYTHON ### What changes were proposed in this pull request? Add some common classes in Python to make it have the same structure as Scala 1. Scala has ClassifierParams/Classifier/ClassificationModel: ``` trait ClassifierParams extends PredictorParams with HasRawPredictionCol abstract class Classifier extends Predictor with ClassifierParams { def setRawPredictionCol } abstract class ClassificationModel extends PredictionModel with ClassifierParams { def setRawPredictionCol } ``` This PR makes Python has the following: ``` class JavaClassifierParams(HasRawPredictionCol, JavaPredictorParams): pass class JavaClassifier(JavaPredictor, JavaClassifierParams): def setRawPredictionCol class JavaClassificationModel(JavaPredictionModel, JavaClassifierParams): def setRawPredictionCol ``` 2. Scala has ProbabilisticClassifierParams/ProbabilisticClassifier/ProbabilisticClassificationModel: ``` trait ProbabilisticClassifierParams extends ClassifierParams with HasProbabilityCol with HasThresholds abstract class ProbabilisticClassifier extends Classifier with ProbabilisticClassifierParams { def setProbabilityCol def setThresholds } abstract class ProbabilisticClassificationModel extends ClassificationModel with ProbabilisticClassifierParams { def setProbabilityCol def setThresholds } ``` This PR makes Python have the following: ``` class JavaProbabilisticClassifierParams(HasProbabilityCol, HasThresholds, JavaClassifierParams): pass class JavaProbabilisticClassifier(JavaClassifier, JavaProbabilisticClassifierParams): def setProbabilityCol def setThresholds class JavaProbabilisticClassificationModel(JavaClassificationModel, JavaProbabilisticClassifierParams): def setProbabilityCol def setThresholds ``` 3. Scala has PredictorParams/Predictor/PredictionModel: ``` trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol abstract class Predictor extends Estimator with PredictorParams { def setLabelCol def setFeaturesCol def setPredictionCol } abstract class PredictionModel extends Model with PredictorParams { def setFeaturesCol def setPredictionCol def numFeatures def predict } ``` This PR makes Python have the following: ``` class JavaPredictorParams(HasLabelCol, HasFeaturesCol, HasPredictionCol): pass class JavaPredictor(JavaEstimator, JavaPredictorParams): def setLabelCol def setFeaturesCol def setPredictionCol class JavaPredictionModel(JavaModel, JavaPredictorParams): def setFeaturesCol def setPredictionCol def numFeatures def predict ``` ### Why are the changes needed? Have parity between Python and Scala ML ### Does this PR introduce any user-facing change? Yes. Add the following changes: ``` LinearSVCModel - get/setFeatureCol - get/setPredictionCol - get/setLabelCol - get/setRawPredictionCol - predict ``` ``` LogisticRegressionModel DecisionTreeClassificationModel RandomForestClassificationModel GBTClassificationModel NaiveBayesModel MultilayerPerceptronClassificationModel - get/setFeatureCol - get/setPredictionCol - get/setLabelCol - get/setRawPredictionCol - get/setProbabilityCol - predict ``` ``` LinearRegressionModel IsotonicRegressionModel DecisionTreeRegressionModel RandomForestRegressionModel GBTRegressionModel AFTSurvivalRegressionModel GeneralizedLinearRegressionModel - get/setFeatureCol - get/setPredictionCol - get/setLabelCol - predict ``` ### How was this patch tested? Add a few doc tests. Closes #25776 from huaxingao/spark-28985. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 212 ++++++++++++++++++++------ python/pyspark/ml/regression.py | 74 +++++---- python/pyspark/ml/tests/test_param.py | 3 +- python/pyspark/ml/util.py | 16 -- python/pyspark/ml/wrapper.py | 74 +++++++++ 5 files changed, 291 insertions(+), 88 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index b6c28d05ea273..e2e313c7f9252 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -26,8 +26,8 @@ DecisionTreeRegressionModel, GBTParams, HasVarianceImpurity, RandomForestParams, \ TreeEnsembleModel from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams -from pyspark.ml.wrapper import JavaWrapper +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ + JavaPredictor, JavaPredictorParams, JavaPredictionModel, JavaWrapper from pyspark.ml.common import inherit_doc, _java2py, _py2java from pyspark.ml.linalg import Vectors from pyspark.sql import DataFrame @@ -47,14 +47,43 @@ 'OneVsRest', 'OneVsRestModel'] +class JavaClassifierParams(HasRawPredictionCol, JavaPredictorParams): + """ + (Private) Java Classifier Params for classification tasks. + """ + pass + + @inherit_doc -class JavaClassificationModel(JavaPredictionModel): +class JavaClassifier(JavaPredictor, JavaClassifierParams): """ - (Private) Java Model produced by a ``Classifier``. + Java Classifier for classification tasks. + Classes are indexed {0, 1, ..., numClasses - 1}. + """ + + @since("3.0.0") + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + + +@inherit_doc +class JavaClassificationModel(JavaPredictionModel, JavaClassifierParams): + """ + Java Model produced by a ``Classifier``. Classes are indexed {0, 1, ..., numClasses - 1}. To be mixed in with class:`pyspark.ml.JavaModel` """ + @since("3.0.0") + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + @property @since("2.1.0") def numClasses(self): @@ -64,10 +93,60 @@ def numClasses(self): return self._call_java("numClasses") +class JavaProbabilisticClassifierParams(HasProbabilityCol, HasThresholds, JavaClassifierParams): + """ + (Private) Java Probabilistic Classifier Params for classification tasks. + """ + pass + + @inherit_doc -class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - HasRegParam, HasTol, HasRawPredictionCol, HasFitIntercept, HasStandardization, - HasWeightCol, HasAggregationDepth, HasThreshold, JavaMLWritable, JavaMLReadable): +class JavaProbabilisticClassifier(JavaClassifier, JavaProbabilisticClassifierParams): + """ + Java Probabilistic Classifier for classification tasks. + """ + + @since("3.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + + @since("3.0.0") + def setThresholds(self, value): + """ + Sets the value of :py:attr:`thresholds`. + """ + return self._set(thresholds=value) + + +@inherit_doc +class JavaProbabilisticClassificationModel(JavaClassificationModel, + JavaProbabilisticClassifierParams): + """ + Java Model produced by a ``ProbabilisticClassifier``. + """ + + @since("3.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + + @since("3.0.0") + def setThresholds(self, value): + """ + Sets the value of :py:attr:`thresholds`. + """ + return self._set(thresholds=value) + + +@inherit_doc +class LinearSVC(JavaClassifier, HasMaxIter, HasRegParam, HasTol, + HasFitIntercept, HasStandardization, HasWeightCol, HasAggregationDepth, + HasThreshold, JavaMLWritable, JavaMLReadable): """ `Linear SVM Classifier `_ @@ -81,6 +160,10 @@ class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, Ha ... Row(label=0.0, features=Vectors.dense(1.0, 2.0, 3.0))]).toDF() >>> svm = LinearSVC(maxIter=5, regParam=0.01) >>> model = svm.fit(df) + >>> model.setPredictionCol("newPrediction") + LinearSVC... + >>> model.getPredictionCol() + 'newPrediction' >>> model.coefficients DenseVector([0.0, -0.2792, -0.1833]) >>> model.intercept @@ -90,8 +173,10 @@ class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, Ha >>> model.numFeatures 3 >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, -1.0, -1.0))]).toDF() + >>> model.predict(test0.head().features) + 1.0 >>> result = model.transform(test0).head() - >>> result.prediction + >>> result.newPrediction 1.0 >>> result.rawPrediction DenseVector([-1.4831, 1.4831]) @@ -156,7 +241,7 @@ def _create_model(self, java_model): return LinearSVCModel(java_model) -class LinearSVCModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable): +class LinearSVCModel(JavaClassificationModel, JavaMLWritable, JavaMLReadable): """ Model fitted by LinearSVC. @@ -181,8 +266,7 @@ def intercept(self): @inherit_doc -class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - HasRegParam, HasTol, HasProbabilityCol, HasRawPredictionCol, +class LogisticRegression(JavaProbabilisticClassifier, HasMaxIter, HasRegParam, HasTol, HasElasticNetParam, HasFitIntercept, HasStandardization, HasThresholds, HasWeightCol, HasAggregationDepth, JavaMLWritable, JavaMLReadable): """ @@ -198,6 +282,12 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti ... Row(label=0.0, weight=4.0, features=Vectors.dense(3.0, 3.0))]).toDF() >>> blor = LogisticRegression(regParam=0.01, weightCol="weight") >>> blorModel = blor.fit(bdf) + >>> blorModel.setFeaturesCol("features") + LogisticRegressionModel... + >>> blorModel.setProbabilityCol("newProbability") + LogisticRegressionModel... + >>> blorModel.getProbabilityCol() + 'newProbability' >>> blorModel.coefficients DenseVector([-1.080..., -0.646...]) >>> blorModel.intercept @@ -211,10 +301,12 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> mlorModel.interceptVector DenseVector([0.04..., -0.42..., 0.37...]) >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, 1.0))]).toDF() + >>> blorModel.predict(test0.head().features) + 1.0 >>> result = blorModel.transform(test0).head() >>> result.prediction 1.0 - >>> result.probability + >>> result.newProbability DenseVector([0.02..., 0.97...]) >>> result.rawPrediction DenseVector([-3.54..., 3.54...]) @@ -481,7 +573,7 @@ def getUpperBoundsOnIntercepts(self): return self.getOrDefault(self.upperBoundsOnIntercepts) -class LogisticRegressionModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable, +class LogisticRegressionModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by LogisticRegression. @@ -872,8 +964,7 @@ def getImpurity(self): @inherit_doc -class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeightCol, - HasPredictionCol, HasProbabilityCol, HasRawPredictionCol, +class DecisionTreeClassifier(JavaProbabilisticClassifier, HasWeightCol, DecisionTreeParams, TreeClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable): """ @@ -892,6 +983,10 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeig >>> td = si_model.transform(df) >>> dt = DecisionTreeClassifier(maxDepth=2, labelCol="indexed", leafCol="leafId") >>> model = dt.fit(td) + >>> model.getLabelCol() + 'indexed' + >>> model.setFeaturesCol("features") + DecisionTreeClassificationModel... >>> model.numNodes 3 >>> model.depth @@ -905,6 +1000,8 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeig >>> print(model.toDebugString) DecisionTreeClassificationModel (uid=...) of depth 1 with 3 nodes... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1031,8 +1128,8 @@ def setImpurity(self, value): @inherit_doc -class DecisionTreeClassificationModel(DecisionTreeModel, JavaClassificationModel, JavaMLWritable, - JavaMLReadable): +class DecisionTreeClassificationModel(DecisionTreeModel, JavaProbabilisticClassificationModel, + JavaMLWritable, JavaMLReadable): """ Model fitted by DecisionTreeClassifier. @@ -1062,9 +1159,8 @@ def featureImportances(self): @inherit_doc -class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasSeed, - HasRawPredictionCol, HasProbabilityCol, - RandomForestParams, TreeClassifierParams, HasCheckpointInterval, +class RandomForestClassifier(JavaProbabilisticClassifier, HasSeed, RandomForestParams, + TreeClassifierParams, HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ `Random Forest `_ @@ -1085,17 +1181,27 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> rf = RandomForestClassifier(numTrees=3, maxDepth=2, labelCol="indexed", seed=42, ... leafCol="leafId") >>> model = rf.fit(td) + >>> model.getLabelCol() + 'indexed' + >>> model.setFeaturesCol("features") + RandomForestClassificationModel... + >>> model.setRawPredictionCol("newRawPrediction") + RandomForestClassificationModel... + >>> model.getRawPredictionCol() + 'newRawPrediction' >>> model.featureImportances SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0, 1.0]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 >>> numpy.argmax(result.probability) 0 - >>> numpy.argmax(result.rawPrediction) + >>> numpy.argmax(result.newRawPrediction) 0 >>> result.leafId DenseVector([0.0, 0.0, 0.0]) @@ -1231,8 +1337,8 @@ def setFeatureSubsetStrategy(self, value): return self._set(featureSubsetStrategy=value) -class RandomForestClassificationModel(TreeEnsembleModel, JavaClassificationModel, JavaMLWritable, - JavaMLReadable): +class RandomForestClassificationModel(TreeEnsembleModel, JavaProbabilisticClassificationModel, + JavaMLWritable, JavaMLReadable): """ Model fitted by RandomForestClassifier. @@ -1284,9 +1390,8 @@ def getLossType(self): @inherit_doc -class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - GBTClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, - JavaMLReadable): +class GBTClassifier(JavaProbabilisticClassifier, GBTClassifierParams, HasCheckpointInterval, + HasSeed, JavaMLWritable, JavaMLReadable): """ `Gradient-Boosted Trees (GBTs) `_ learning algorithm for classification. @@ -1318,11 +1423,21 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol >>> gbt.getFeatureSubsetStrategy() 'all' >>> model = gbt.fit(td) + >>> model.getLabelCol() + 'indexed' + >>> model.setFeaturesCol("features") + GBTClassificationModel... + >>> model.setThresholds([0.3, 0.7]) + GBTClassificationModel... + >>> model.getThresholds() + [0.3, 0.7] >>> model.featureImportances SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1485,8 +1600,8 @@ def setValidationIndicatorCol(self, value): return self._set(validationIndicatorCol=value) -class GBTClassificationModel(TreeEnsembleModel, JavaClassificationModel, JavaMLWritable, - JavaMLReadable): +class GBTClassificationModel(TreeEnsembleModel, JavaProbabilisticClassificationModel, + JavaMLWritable, JavaMLReadable): """ Model fitted by GBTClassifier. @@ -1527,8 +1642,8 @@ def evaluateEachIteration(self, dataset): @inherit_doc -class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, - HasRawPredictionCol, HasThresholds, HasWeightCol, JavaMLWritable, JavaMLReadable): +class NaiveBayes(JavaProbabilisticClassifier, HasThresholds, HasWeightCol, + JavaMLWritable, JavaMLReadable): """ Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB @@ -1547,11 +1662,19 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H ... Row(label=1.0, weight=1.0, features=Vectors.dense([1.0, 0.0]))]) >>> nb = NaiveBayes(smoothing=1.0, modelType="multinomial", weightCol="weight") >>> model = nb.fit(df) + >>> model.setFeaturesCol("features") + NaiveBayes_... + >>> model.setLabelCol("newLabel") + NaiveBayes_... + >>> model.getLabelCol() + 'newLabel' >>> model.pi DenseVector([-0.81..., -0.58...]) >>> model.theta DenseMatrix(2, 2, [-0.91..., -0.51..., -0.40..., -1.09...], 1) >>> test0 = sc.parallelize([Row(features=Vectors.dense([1.0, 0.0]))]).toDF() + >>> model.predict(test0.head().features) + 1.0 >>> result = model.transform(test0).head() >>> result.prediction 1.0 @@ -1651,7 +1774,7 @@ def getModelType(self): return self.getOrDefault(self.modelType) -class NaiveBayesModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable): +class NaiveBayesModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): """ Model fitted by NaiveBayes. @@ -1676,10 +1799,8 @@ def theta(self): @inherit_doc -class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasMaxIter, HasTol, HasSeed, HasStepSize, HasSolver, - JavaMLWritable, JavaMLReadable, HasProbabilityCol, - HasRawPredictionCol): +class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, HasMaxIter, HasTol, HasSeed, + HasStepSize, HasSolver, JavaMLWritable, JavaMLReadable): """ Classifier trainer based on the Multilayer Perceptron. Each layer has sigmoid activation function, output layer has softmax. @@ -1694,6 +1815,8 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) >>> mlp = MultilayerPerceptronClassifier(maxIter=100, layers=[2, 2, 2], blockSize=1, seed=123) >>> model = mlp.fit(df) + >>> model.setFeaturesCol("features") + MultilayerPerceptronClassifier... >>> model.layers [2, 2, 2] >>> model.weights.size @@ -1701,6 +1824,8 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, >>> testDF = spark.createDataFrame([ ... (Vectors.dense([1.0, 0.0]),), ... (Vectors.dense([0.0, 0.0]),)], ["features"]) + >>> model.predict(testDF.head().features) + 1.0 >>> model.transform(testDF).select("features", "prediction").show() +---------+----------+ | features|prediction| @@ -1839,7 +1964,7 @@ def getInitialWeights(self): return self.getOrDefault(self.initialWeights) -class MultilayerPerceptronClassificationModel(JavaModel, JavaClassificationModel, JavaMLWritable, +class MultilayerPerceptronClassificationModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): """ Model fitted by MultilayerPerceptronClassifier. @@ -1864,8 +1989,7 @@ def weights(self): return self._call_java("weights") -class OneVsRestParams(HasFeaturesCol, HasLabelCol, HasWeightCol, HasPredictionCol, - HasRawPredictionCol): +class OneVsRestParams(JavaClassifierParams, HasWeightCol): """ Parameters for OneVsRest and OneVsRestModel. """ @@ -1897,6 +2021,8 @@ class OneVsRest(Estimator, OneVsRestParams, HasParallelism, JavaMLReadable, Java >>> ovr = OneVsRest(classifier=lr) >>> ovr.getRawPredictionCol() 'rawPrediction' + >>> ovr.setPredictionCol("newPrediction") + OneVsRest... >>> model = ovr.fit(df) >>> model.models[0].coefficients DenseVector([0.5..., -1.0..., 3.4..., 4.2...]) @@ -1907,21 +2033,21 @@ class OneVsRest(Estimator, OneVsRestParams, HasParallelism, JavaMLReadable, Java >>> [x.intercept for x in model.models] [-2.7..., -2.5..., -1.3...] >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, 0.0, 1.0, 1.0))]).toDF() - >>> model.transform(test0).head().prediction + >>> model.transform(test0).head().newPrediction 0.0 >>> test1 = sc.parallelize([Row(features=Vectors.sparse(4, [0], [1.0]))]).toDF() - >>> model.transform(test1).head().prediction + >>> model.transform(test1).head().newPrediction 2.0 >>> test2 = sc.parallelize([Row(features=Vectors.dense(0.5, 0.4, 0.3, 0.2))]).toDF() - >>> model.transform(test2).head().prediction + >>> model.transform(test2).head().newPrediction 0.0 >>> model_path = temp_path + "/ovr_model" >>> model.save(model_path) >>> model2 = OneVsRestModel.load(model_path) - >>> model2.transform(test0).head().prediction + >>> model2.transform(test0).head().newPrediction 0.0 >>> model.transform(test2).columns - ['features', 'rawPrediction', 'prediction'] + ['features', 'rawPrediction', 'newPrediction'] .. versionadded:: 2.0.0 """ diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 2d1d1272c17f8..f2bcc662030c6 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -20,7 +20,8 @@ from pyspark import since, keyword_only from pyspark.ml.param.shared import * from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaWrapper +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ + JavaPredictor, JavaPredictionModel, JavaWrapper from pyspark.ml.common import inherit_doc from pyspark.sql import DataFrame @@ -37,10 +38,9 @@ @inherit_doc -class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - HasRegParam, HasTol, HasElasticNetParam, HasFitIntercept, - HasStandardization, HasSolver, HasWeightCol, HasAggregationDepth, HasLoss, - JavaMLWritable, JavaMLReadable): +class LinearRegression(JavaPredictor, HasMaxIter, HasRegParam, HasTol, HasElasticNetParam, + HasFitIntercept, HasStandardization, HasSolver, HasWeightCol, + HasAggregationDepth, HasLoss, JavaMLWritable, JavaMLReadable): """ Linear regression. @@ -66,15 +66,21 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction ... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) >>> lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight") >>> model = lr.fit(df) + >>> model.setFeaturesCol("features") + LinearRegression... + >>> model.setPredictionCol("newPrediction") + LinearRegression... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) - >>> abs(model.transform(test0).head().prediction - (-1.0)) < 0.001 + >>> abs(model.predict(test0.head().features) - (-1.0)) < 0.001 + True + >>> abs(model.transform(test0).head().newPrediction - (-1.0)) < 0.001 True >>> abs(model.coefficients[0] - 1.0) < 0.001 True >>> abs(model.intercept - 0.0) < 0.001 True >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) - >>> abs(model.transform(test1).head().prediction - 1.0) < 0.001 + >>> abs(model.transform(test1).head().newPrediction - 1.0) < 0.001 True >>> lr.setParams("vector") Traceback (most recent call last): @@ -161,7 +167,7 @@ def getEpsilon(self): return self.getOrDefault(self.epsilon) -class LinearRegressionModel(JavaModel, JavaPredictionModel, GeneralJavaMLWritable, JavaMLReadable, +class LinearRegressionModel(JavaPredictionModel, GeneralJavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`LinearRegression`. @@ -460,8 +466,7 @@ def totalIterations(self): @inherit_doc -class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasWeightCol, JavaMLWritable, JavaMLReadable): +class IsotonicRegression(JavaPredictor, HasWeightCol, JavaMLWritable, JavaMLReadable): """ Currently implemented using parallelized pool adjacent violators algorithm. Only univariate (single feature) algorithm supported. @@ -472,6 +477,12 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> ir = IsotonicRegression() >>> model = ir.fit(df) + >>> model.setFeaturesCol("features") + IsotonicRegression... + >>> model.setLabelCol("newLabel") + IsotonicRegression... + >>> model.getLabelCol() + 'newLabel' >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 @@ -555,7 +566,7 @@ def getFeatureIndex(self): return self.getOrDefault(self.featureIndex) -class IsotonicRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): +class IsotonicRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable): """ Model fitted by :class:`IsotonicRegression`. @@ -804,8 +815,7 @@ def getLossType(self): @inherit_doc -class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeightCol, - HasPredictionCol, DecisionTreeParams, TreeRegressorParams, +class DecisionTreeRegressor(JavaPredictor, HasWeightCol, DecisionTreeParams, TreeRegressorParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable, HasVarianceCol): """ @@ -828,6 +838,8 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeigh >>> model.numFeatures 1 >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -950,7 +962,7 @@ def setImpurity(self, value): @inherit_doc -class DecisionTreeModel(JavaModel, JavaPredictionModel): +class DecisionTreeModel(JavaPredictionModel): """ Abstraction for Decision Tree models. @@ -1052,9 +1064,8 @@ def featureImportances(self): @inherit_doc -class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasSeed, - RandomForestParams, TreeRegressorParams, HasCheckpointInterval, - JavaMLWritable, JavaMLReadable): +class RandomForestRegressor(JavaPredictor, HasSeed, RandomForestParams, TreeRegressorParams, + HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ `Random Forest `_ learning algorithm for regression. @@ -1072,6 +1083,8 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> allclose(model.treeWeights, [1.0, 1.0]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1240,9 +1253,8 @@ def featureImportances(self): @inherit_doc -class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - GBTRegressorParams, HasCheckpointInterval, HasSeed, JavaMLWritable, - JavaMLReadable): +class GBTRegressor(JavaPredictor, GBTRegressorParams, HasCheckpointInterval, HasSeed, + JavaMLWritable, JavaMLReadable): """ `Gradient-Boosted Trees (GBTs) `_ learning algorithm for regression. @@ -1259,6 +1271,8 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, >>> print(gbt.getFeatureSubsetStrategy()) all >>> model = gbt.fit(df) + >>> model.setFeaturesCol("features") + GBTRegressionModel... >>> model.featureImportances SparseVector(1, {0: 1.0}) >>> model.numFeatures @@ -1266,6 +1280,8 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1465,9 +1481,8 @@ def evaluateEachIteration(self, dataset, loss): @inherit_doc -class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasFitIntercept, HasMaxIter, HasTol, HasAggregationDepth, - JavaMLWritable, JavaMLReadable): +class AFTSurvivalRegression(JavaPredictor, HasFitIntercept, HasMaxIter, HasTol, + HasAggregationDepth, JavaMLWritable, JavaMLReadable): """ Accelerated Failure Time (AFT) Model Survival Regression @@ -1482,6 +1497,8 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi ... (1e-40, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) >>> aftsr = AFTSurvivalRegression() >>> model = aftsr.fit(df) + >>> model.setFeaturesCol("features") + AFTSurvivalRegression... >>> model.predict(Vectors.dense(6.3)) 1.0 >>> model.predictQuantiles(Vectors.dense(6.3)) @@ -1607,7 +1624,7 @@ def getQuantilesCol(self): return self.getOrDefault(self.quantilesCol) -class AFTSurvivalRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): +class AFTSurvivalRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable): """ Model fitted by :class:`AFTSurvivalRegression`. @@ -1654,9 +1671,8 @@ def predict(self, features): @inherit_doc -class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, HasPredictionCol, - HasFitIntercept, HasMaxIter, HasTol, HasRegParam, HasWeightCol, - HasSolver, JavaMLWritable, JavaMLReadable): +class GeneralizedLinearRegression(JavaPredictor, HasFitIntercept, HasMaxIter, HasTol, HasRegParam, + HasWeightCol, HasSolver, JavaMLWritable, JavaMLReadable): """ Generalized Linear Regression. @@ -1686,6 +1702,8 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha ... (2.0, Vectors.dense(1.0, 1.0)),], ["label", "features"]) >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity", linkPredictionCol="p") >>> model = glr.fit(df) + >>> model.setFeaturesCol("features") + GeneralizedLinearRegression... >>> transformed = model.transform(df) >>> abs(transformed.head().prediction - 1.5) < 0.001 True @@ -1861,7 +1879,7 @@ def getOffsetCol(self): return self.getOrDefault(self.offsetCol) -class GeneralizedLinearRegressionModel(JavaModel, JavaPredictionModel, JavaMLWritable, +class GeneralizedLinearRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`GeneralizedLinearRegression`. diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 72b146054ca44..f12310c7d8eba 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -352,7 +352,8 @@ def test_java_params(self): for module in modules: for name, cls in inspect.getmembers(module, inspect.isclass): if not name.endswith('Model') and not name.endswith('Params') \ - and issubclass(cls, JavaParams) and not inspect.isabstract(cls): + and issubclass(cls, JavaParams) and not inspect.isabstract(cls) \ + and not name.startswith('Java'): # NOTE: disable check_params_exist until there is parity with Scala API check_params(self, cls(), check_params_exist=False) diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index debaf38e0896e..81b785e71f302 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -342,22 +342,6 @@ def read(cls): return JavaMLReader(cls) -@inherit_doc -class JavaPredictionModel(): - """ - (Private) Java Model for prediction tasks (regression and classification). - To be mixed in with class:`pyspark.ml.JavaModel` - """ - - @property - @since("2.1.0") - def numFeatures(self): - """ - Returns the number of features the model was trained on. If unknown, returns -1 - """ - return self._call_java("numFeatures") - - @inherit_doc class DefaultParamsWritable(MLWritable): """ diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 9bb1262a54500..3137db7181c07 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -20,10 +20,12 @@ if sys.version >= '3': xrange = range +from pyspark import since from pyspark import SparkContext from pyspark.sql import DataFrame from pyspark.ml import Estimator, Transformer, Model from pyspark.ml.param import Params +from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol from pyspark.ml.util import _jvm from pyspark.ml.common import inherit_doc, _java2py, _py2java @@ -361,3 +363,75 @@ def __init__(self, java_model=None): self._create_params_from_java() self._resetUid(java_model.uid()) + + +@inherit_doc +class JavaPredictorParams(HasLabelCol, HasFeaturesCol, HasPredictionCol): + """ + (Private) Trait for parameters for prediction (regression and classification) + """ + pass + + +@inherit_doc +class JavaPredictor(JavaEstimator, JavaPredictorParams): + """ + (Private) Java Estimator for prediction tasks (regression and classification). + """ + + @since("3.0.0") + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + +@inherit_doc +class JavaPredictionModel(JavaModel, JavaPredictorParams): + """ + (Private) Java Model for prediction tasks (regression and classification). + """ + + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @property + @since("2.1.0") + def numFeatures(self): + """ + Returns the number of features the model was trained on. If unknown, returns -1 + """ + return self._call_java("numFeatures") + + @since("3.0.0") + def predict(self, value): + """ + Predict label for the given features. + """ + return self._call_java("predict", value) From c5d8a51f3b956c226d38f8fdc9ec644839e8b2f2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 19 Sep 2019 11:37:42 -0700 Subject: [PATCH 51/87] [MINOR][BUILD] Fix about 15 misc build warnings ### What changes were proposed in this pull request? This addresses about 15 miscellaneous warnings that appear in the current build. ### Why are the changes needed? No functional changes, it just slightly reduces the amount of extra warning output. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests, run manually. Closes #25852 from srowen/BuildWarnings. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/util/kvstore/InMemoryStore.java | 2 +- .../java/org/apache/spark/util/kvstore/KVTypeInfo.java | 6 +++--- .../spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 6 +++--- .../spark/deploy/history/HistoryServerSuite.scala | 4 +--- .../scala/org/apache/spark/util/KeyLockSuite.scala | 2 +- .../apache/spark/repl/ExecutorClassLoaderSuite.scala | 3 ++- .../spark/sql/catalyst/trees/TreeNodeSuite.scala | 10 +++++----- 7 files changed, 16 insertions(+), 17 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java index 6af45aec3c7b2..b33c53871c32f 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java @@ -252,7 +252,7 @@ private static Predicate getPredicate( return (value) -> set.contains(indexValueForEntity(getter, value)); } else { - HashSet set = new HashSet<>(values.size()); + HashSet> set = new HashSet<>(values.size()); for (Object key : values) { set.add(asKey(key)); } diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java index b8c5fab8709ed..d2a26982d8703 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java @@ -124,7 +124,7 @@ interface Accessor { Object get(Object instance) throws ReflectiveOperationException; - Class getType(); + Class getType(); } private class FieldAccessor implements Accessor { @@ -141,7 +141,7 @@ public Object get(Object instance) throws ReflectiveOperationException { } @Override - public Class getType() { + public Class getType() { return field.getType(); } } @@ -160,7 +160,7 @@ public Object get(Object instance) throws ReflectiveOperationException { } @Override - public Class getType() { + public Class getType() { return method.getReturnType(); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 1022111897a49..4f660a13f7400 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -172,9 +172,9 @@ public void setUp() throws IOException { when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); } - private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { + private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); - return new UnsafeShuffleWriter( + return new UnsafeShuffleWriter<>( blockManager, taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), @@ -533,7 +533,7 @@ public void testPeakMemoryUsed() throws Exception { final long numRecordsPerPage = pageSizeBytes / recordLengthBytes; taskMemoryManager = spy(taskMemoryManager); when(taskMemoryManager.pageSizeBytes()).thenReturn(pageSizeBytes); - final UnsafeShuffleWriter writer = new UnsafeShuffleWriter( + final UnsafeShuffleWriter writer = new UnsafeShuffleWriter<>( blockManager, taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 71a127bd4b9f6..e3cfec90ee81b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -26,7 +26,6 @@ import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpSe import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import com.gargoylesoftware.htmlunit.BrowserVersion import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -365,8 +364,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers contextHandler.addServlet(holder, "/") server.attachHandler(contextHandler) - implicit val webDriver: WebDriver = - new HtmlUnitDriver(BrowserVersion.INTERNET_EXPLORER_11, true) + implicit val webDriver: WebDriver = new HtmlUnitDriver(true) try { val url = s"http://localhost:$port" diff --git a/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala b/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala index 2169a0e4d442f..6888e492a8d33 100644 --- a/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala @@ -49,7 +49,7 @@ class KeyLockSuite extends SparkFunSuite with TimeLimits { @volatile var e: Throwable = null val threads = (0 until numThreads).map { i => new Thread() { - override def run(): Unit = try { + override def run(): Unit = { latch.await(foreverMs, TimeUnit.MILLISECONDS) keyLock.withLock(keys(i)) { var cur = numThreadsHoldingLock.get() diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 1d2e8acff9a3a..6c694804e451b 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -272,7 +272,8 @@ class ExecutorClassLoaderSuite assert(e.getMessage.contains("ThisIsAClassName")) // RemoteClassLoaderError must not be LinkageError nor ClassNotFoundException. Otherwise, // JVM will cache it and doesn't retry to load a class. - assert(!e.isInstanceOf[LinkageError] && !e.isInstanceOf[ClassNotFoundException]) + assert(!(classOf[LinkageError].isAssignableFrom(e.getClass))) + assert(!(classOf[ClassNotFoundException].isAssignableFrom(e.getClass))) } finally { rpcEnv.shutdown() rpcEnv.awaitTermination() 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 fbaa5527a7058..828706743c5b5 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 @@ -680,12 +680,12 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { } test("clone") { - def assertDifferentInstance(before: AnyRef, after: AnyRef): Unit = { + def assertDifferentInstance[T <: TreeNode[T]](before: TreeNode[T], after: TreeNode[T]): Unit = { assert(before.ne(after) && before == after) - before.asInstanceOf[TreeNode[_]].children.zip( - after.asInstanceOf[TreeNode[_]].children).foreach { - case (beforeChild: AnyRef, afterChild: AnyRef) => - assertDifferentInstance(beforeChild, afterChild) + before.children.zip(after.children).foreach { case (beforeChild, afterChild) => + assertDifferentInstance( + beforeChild.asInstanceOf[TreeNode[T]], + afterChild.asInstanceOf[TreeNode[T]]) } } From eee2e026bb4019020b492c54ba51a97df42b13c2 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 19 Sep 2019 11:47:47 -0700 Subject: [PATCH 52/87] [SPARK-29165][SQL][TEST] Set log level of log generated code as ERROR in case of compile error on generated code in UT ### What changes were proposed in this pull request? This patch proposes to change the log level of logging generated code in case of compile error being occurred in UT. This would help to investigate compilation issue of generated code easier, as currently we got exception message of line number but there's no generated code being logged actually (as in most cases of UT the threshold of log level is at least WARN). ### Why are the changes needed? This would help investigating issue on compilation error for generated code in UT. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #25835 from HeartSaVioR/MINOR-always-log-generated-code-on-fail-to-compile-in-unit-testing. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../expressions/codegen/CodeGenerator.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 1d010ff5abfcf..68ddec9fc8d00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1369,20 +1369,27 @@ object CodeGenerator extends Logging { case e: InternalCompilerException => val msg = s"failed to compile: $e" logError(msg, e) - val maxLines = SQLConf.get.loggingMaxLinesForCodegen - logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + logGeneratedCode(code) throw new InternalCompilerException(msg, e) case e: CompileException => val msg = s"failed to compile: $e" logError(msg, e) - val maxLines = SQLConf.get.loggingMaxLinesForCodegen - logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + logGeneratedCode(code) throw new CompileException(msg, e.getLocation) } (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], codeStats) } + private def logGeneratedCode(code: CodeAndComment): Unit = { + val maxLines = SQLConf.get.loggingMaxLinesForCodegen + if (Utils.isTesting) { + logError(s"\n${CodeFormatter.format(code, maxLines)}") + } else { + logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + } + } + /** * Returns the bytecode statistics (max method bytecode size, max constant pool size, and * # of inner classes) of generated classes by inspecting Janino classes. From d3eb4c94cc0d927c7bab310b05e1bb9427a7d67a Mon Sep 17 00:00:00 2001 From: shivusondur Date: Thu, 19 Sep 2019 13:04:17 -0700 Subject: [PATCH 53/87] [SPARK-28822][DOC][SQL] Document USE DATABASE in SQL Reference ### What changes were proposed in this pull request? Added document reference for USE databse sql command ### Why are the changes needed? For USE database command usage ### Does this PR introduce any user-facing change? It is adding the USE database sql command refernce information in the doc ### How was this patch tested? Attached the test snap ![image](https://user-images.githubusercontent.com/7912929/65170499-7242a380-da66-11e9-819c-76df62c86c5a.png) Closes #25572 from shivusondur/jiraUSEDaBa1. Lead-authored-by: shivusondur Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/_data/menu-sql.yaml | 2 + docs/sql-ref-syntax-qry-select-usedb.md | 60 +++++++++++++++++++++++++ 2 files changed, 62 insertions(+) create mode 100644 docs/sql-ref-syntax-qry-select-usedb.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 07ca150a1523f..edcdad4ee7db5 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -133,6 +133,8 @@ url: sql-ref-syntax-qry-select-limit.html - text: Set operations url: sql-ref-syntax-qry-select-setops.html + - text: USE database + url: sql-ref-syntax-qry-select-usedb.html - text: Common Table Expression(CTE) url: sql-ref-syntax-qry-select-cte.html - text: Subqueries diff --git a/docs/sql-ref-syntax-qry-select-usedb.md b/docs/sql-ref-syntax-qry-select-usedb.md new file mode 100644 index 0000000000000..92ac91ac51769 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-usedb.md @@ -0,0 +1,60 @@ +--- +layout: global +title: USE Database +displayTitle: USE Database +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 +`USE` statement is used to set the current database. After the current database is set, +the unqualified database artifacts such as tables, functions and views that are +referenced by SQLs are resolved from the current database. +The default database name is 'default'. + +### Syntax +{% highlight sql %} +USE database_name +{% endhighlight %} + +### Parameter + +
    +
    database_name
    +
    + Name of the database will be used. If the database does not exist, an exception will be thrown. +
    +
    + +### Example +{% highlight sql %} +-- Use the 'userdb' which exists. +USE userdb; ++---------+--+ +| Result | ++---------+--+ ++---------+--+ + +-- Use the 'userdb1' which doesn't exist +USE userdb1; +Error: org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'userdb1' not found;(state=,code=0) +{% endhighlight %} + +### Related statements. +- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) +- [CREATE TABLE ](sql-ref-syntax-ddl-create-table.html) + From 2c775f418f5fae4dbf3adfbb5ea99cd030918d41 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 19 Sep 2019 13:32:09 -0700 Subject: [PATCH 54/87] [SPARK-28612][SQL] Add DataFrameWriterV2 API ## What changes were proposed in this pull request? This adds a new write API as proposed in the [SPIP to standardize logical plans](https://issues.apache.org/jira/browse/SPARK-23521). This new API: * Uses clear verbs to execute writes, like `append`, `overwrite`, `create`, and `replace` that correspond to the new logical plans. * Only creates v2 logical plans so the behavior is always consistent. * Does not allow table configuration options for operations that cannot change table configuration. For example, `partitionedBy` can only be called when the writer executes `create` or `replace`. Here are a few example uses of the new API: ```scala df.writeTo("catalog.db.table").append() df.writeTo("catalog.db.table").overwrite($"date" === "2019-06-01") df.writeTo("catalog.db.table").overwritePartitions() df.writeTo("catalog.db.table").asParquet.create() df.writeTo("catalog.db.table").partitionedBy(days($"ts")).createOrReplace() df.writeTo("catalog.db.table").using("abc").replace() ``` ## How was this patch tested? Added `DataFrameWriterV2Suite` that tests the new write API. Existing tests for v2 plans. Closes #25681 from rdblue/SPARK-28612-add-data-frame-writer-v2. Authored-by: Ryan Blue Signed-off-by: Burak Yavuz --- .../expressions/PartitionTransforms.scala | 77 +++ .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../plans/logical/basicLogicalOperators.scala | 47 +- .../v2/DataSourceV2Implicits.scala | 9 + .../spark/sql/connector/InMemoryTable.scala | 5 +- .../apache/spark/sql/DataFrameWriter.scala | 11 +- .../apache/spark/sql/DataFrameWriterV2.scala | 367 +++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 28 + .../datasources/v2/DataSourceV2Strategy.scala | 17 +- .../datasources/v2/TableCapabilityCheck.scala | 6 +- .../org/apache/spark/sql/functions.scala | 58 ++ .../spark/sql/JavaDataFrameWriterV2Suite.java | 112 ++++ .../spark/sql/DataFrameWriterV2Suite.scala | 507 ++++++++++++++++++ 13 files changed, 1217 insertions(+), 33 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala new file mode 100644 index 0000000000000..e48fd8adaef09 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.types.{DataType, IntegerType} + +/** + * Base class for expressions that are converted to v2 partition transforms. + * + * Subclasses represent abstract transform functions with concrete implementations that are + * determined by data source implementations. Because the concrete implementation is not known, + * these expressions are [[Unevaluable]]. + * + * These expressions are used to pass transformations from the DataFrame API: + * + * {{{ + * df.writeTo("catalog.db.table").partitionedBy($"category", days($"timestamp")).create() + * }}} + */ +abstract class PartitionTransformExpression extends Expression with Unevaluable { + override def nullable: Boolean = true +} + +/** + * Expression for the v2 partition transform years. + */ +case class Years(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform months. + */ +case class Months(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform days. + */ +case class Days(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform hours. + */ +case class Hours(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform bucket. + */ +case class Bucket(numBuckets: Literal, child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(numBuckets, child) +} 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 76e59fa785426..7c81185388d02 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 @@ -2564,7 +2564,7 @@ class Analyzer( */ object ResolveOutputRelation extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { - case append @ AppendData(table, query, isByName) + case append @ AppendData(table, query, _, isByName) if table.resolved && query.resolved && !append.outputResolved => val projection = TableOutputResolver.resolveOutputColumns( @@ -2576,7 +2576,7 @@ class Analyzer( append } - case overwrite @ OverwriteByExpression(table, _, query, isByName) + case overwrite @ OverwriteByExpression(table, _, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => val projection = TableOutputResolver.resolveOutputColumns( @@ -2588,7 +2588,7 @@ class Analyzer( overwrite } - case overwrite @ OverwritePartitionsDynamic(table, query, isByName) + case overwrite @ OverwritePartitionsDynamic(table, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => val projection = TableOutputResolver.resolveOutputColumns( 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 2b6378e7c7268..9356d7707104e 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 @@ -488,7 +488,7 @@ case class ReplaceTableAsSelect( override def tableSchema: StructType = query.schema override def children: Seq[LogicalPlan] = Seq(query) - override lazy val resolved: Boolean = { + 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 @@ -506,15 +506,22 @@ case class ReplaceTableAsSelect( case class AppendData( table: NamedRelation, query: LogicalPlan, + writeOptions: Map[String, String], isByName: Boolean) extends V2WriteCommand object AppendData { - def byName(table: NamedRelation, df: LogicalPlan): AppendData = { - new AppendData(table, df, isByName = true) + 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): AppendData = { - new AppendData(table, query, isByName = false) + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, query, writeOptions, isByName = false) } } @@ -525,19 +532,26 @@ 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): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, df, isByName = true) + 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): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, query, isByName = false) + table: NamedRelation, + query: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, query, writeOptions, isByName = false) } } @@ -547,15 +561,22 @@ object OverwriteByExpression { case class OverwritePartitionsDynamic( table: NamedRelation, query: LogicalPlan, + writeOptions: Map[String, String], isByName: Boolean) extends V2WriteCommand object OverwritePartitionsDynamic { - def byName(table: NamedRelation, df: LogicalPlan): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, df, isByName = true) + 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): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, query, isByName = false) + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, query, writeOptions, isByName = false) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index 74842e762b401..86ef867eca547 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.execution.datasources.v2 +import scala.collection.JavaConverters._ + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Implicits { implicit class TableHelper(table: Table) { @@ -53,4 +56,10 @@ object DataSourceV2Implicits { def supportsAny(capabilities: TableCapability*): Boolean = capabilities.exists(supports) } + + implicit class OptionsHelper(options: Map[String, String]) { + def asOptions: CaseInsensitiveStringMap = { + new CaseInsensitiveStringMap(options.asJava) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 04ae27df94535..414f9d5834868 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -41,8 +41,11 @@ class InMemoryTable( override val properties: util.Map[String, String]) extends Table with SupportsRead with SupportsWrite with SupportsDelete { + private val allowUnsupportedTransforms = + properties.getOrDefault("allow-unsupported-transforms", "false").toBoolean + partitioning.foreach { t => - if (!t.isInstanceOf[IdentityTransform]) { + if (!t.isInstanceOf[IdentityTransform] && !allowUnsupportedTransforms) { throw new IllegalArgumentException(s"Transform $t must be IdentityTransform") } } 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 58acfb836b305..289a976c6db9e 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 @@ -271,13 +271,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { modeForDSV2 match { case SaveMode.Append => runCommand(df.sparkSession, "save") { - AppendData.byName(relation, df.logicalPlan) + AppendData.byName(relation, df.logicalPlan, extraOptions.toMap) } case SaveMode.Overwrite if table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER) => // truncate the table runCommand(df.sparkSession, "save") { - OverwriteByExpression.byName(relation, df.logicalPlan, Literal(true)) + OverwriteByExpression.byName( + relation, df.logicalPlan, Literal(true), extraOptions.toMap) } case other => @@ -382,7 +383,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val command = modeForDSV2 match { case SaveMode.Append => - AppendData.byPosition(table, df.logicalPlan) + AppendData.byPosition(table, df.logicalPlan, extraOptions.toMap) case SaveMode.Overwrite => val conf = df.sparkSession.sessionState.conf @@ -390,9 +391,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC if (dynamicPartitionOverwrite) { - OverwritePartitionsDynamic.byPosition(table, df.logicalPlan) + OverwritePartitionsDynamic.byPosition(table, df.logicalPlan, extraOptions.toMap) } else { - OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true)) + OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true), extraOptions.toMap) } case other => 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 new file mode 100644 index 0000000000000..d3ab33919a7bb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -0,0 +1,367 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Bucket, Days, Hours, Literal, Months, Years} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, Transform} +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.IntegerType + +/** + * Interface used to write a [[org.apache.spark.sql.Dataset]] to external storage using the v2 API. + * + * @since 3.0.0 + */ +@Experimental +final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) + extends CreateTableWriter[T] { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ + import df.sparkSession.sessionState.analyzer.CatalogObjectIdentifier + + private val df: DataFrame = ds.toDF() + + private val sparkSession = ds.sparkSession + + private val catalogManager = sparkSession.sessionState.analyzer.catalogManager + + 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) + } + + private val logicalPlan = df.queryExecution.logical + + private var provider: Option[String] = None + + private val options = new mutable.HashMap[String, String]() + + private val properties = new mutable.HashMap[String, String]() + + private var partitioning: Option[Seq[Transform]] = None + + override def using(provider: String): CreateTableWriter[T] = { + this.provider = Some(provider) + this + } + + override def option(key: String, value: String): DataFrameWriterV2[T] = { + this.options.put(key, value) + this + } + + override def options(options: scala.collection.Map[String, String]): DataFrameWriterV2[T] = { + options.foreach { + case (key, value) => + this.options.put(key, value) + } + this + } + + override def options(options: java.util.Map[String, String]): DataFrameWriterV2[T] = { + this.options(options.asScala) + this + } + + override def tableProperty(property: String, value: String): DataFrameWriterV2[T] = { + this.properties.put(property, value) + this + } + + @scala.annotation.varargs + override def partitionedBy(column: Column, columns: Column*): CreateTableWriter[T] = { + val asTransforms = (column +: columns).map(_.expr).map { + case Years(attr: Attribute) => + LogicalExpressions.years(attr.name) + case Months(attr: Attribute) => + LogicalExpressions.months(attr.name) + case Days(attr: Attribute) => + LogicalExpressions.days(attr.name) + case Hours(attr: Attribute) => + LogicalExpressions.hours(attr.name) + case Bucket(Literal(numBuckets: Int, IntegerType), attr: Attribute) => + LogicalExpressions.bucket(numBuckets, attr.name) + case attr: Attribute => + LogicalExpressions.identity(attr.name) + case expr => + throw new AnalysisException(s"Invalid partition transformation: ${expr.sql}") + } + + this.partitioning = Some(asTransforms) + this + } + + override def create(): Unit = { + // create and replace could alternatively create ParsedPlan statements, like + // `CreateTableFromDataFrameStatement(UnresolvedRelation(tableName), ...)`, to keep the catalog + // resolution logic in the analyzer. + runCommand("create") { + CreateTableAsSelect( + catalog, + identifier, + partitioning.getOrElse(Seq.empty), + logicalPlan, + properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, + writeOptions = options.toMap, + ignoreIfExists = false) + } + } + + override def replace(): Unit = { + internalReplace(orCreate = false) + } + + override def createOrReplace(): Unit = { + internalReplace(orCreate = true) + } + + + /** + * Append the contents of the data frame to the output table. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be + * validated to ensure it is compatible with the existing table. + * + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + */ + @throws(classOf[NoSuchTableException]) + def append(): Unit = { + val append = loadTable(catalog, identifier) match { + case Some(t) => + AppendData.byName(DataSourceV2Relation.create(t), logicalPlan, options.toMap) + case _ => + throw new NoSuchTableException(identifier) + } + + runCommand("append")(append) + } + + /** + * Overwrite rows matching the given filter condition with the contents of the data frame in + * the output table. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. + * The data frame will be validated to ensure it is compatible with the existing table. + * + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + */ + @throws(classOf[NoSuchTableException]) + def overwrite(condition: Column): Unit = { + val overwrite = loadTable(catalog, identifier) match { + case Some(t) => + OverwriteByExpression.byName( + DataSourceV2Relation.create(t), logicalPlan, condition.expr, options.toMap) + case _ => + throw new NoSuchTableException(identifier) + } + + runCommand("overwrite")(overwrite) + } + + /** + * Overwrite all partition for which the data frame contains at least one row with the contents + * of the data frame in the output table. + * + * This operation is equivalent to Hive's `INSERT OVERWRITE ... PARTITION`, which replaces + * partitions dynamically depending on the contents of the data frame. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be + * validated to ensure it is compatible with the existing table. + * + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + */ + @throws(classOf[NoSuchTableException]) + def overwritePartitions(): Unit = { + val dynamicOverwrite = loadTable(catalog, identifier) match { + case Some(t) => + OverwritePartitionsDynamic.byName( + DataSourceV2Relation.create(t), logicalPlan, options.toMap) + case _ => + throw new NoSuchTableException(identifier) + } + + runCommand("overwritePartitions")(dynamicOverwrite) + } + + /** + * Wrap an action to track the QueryExecution and time cost, then report to the user-registered + * callback functions. + */ + private def runCommand(name: String)(command: LogicalPlan): Unit = { + val qe = sparkSession.sessionState.executePlan(command) + // call `QueryExecution.toRDD` to trigger the execution of commands. + SQLExecution.withNewExecutionId(sparkSession, qe, Some(name))(qe.toRdd) + } + + private def internalReplace(orCreate: Boolean): Unit = { + runCommand("replace") { + ReplaceTableAsSelect( + catalog, + identifier, + partitioning.getOrElse(Seq.empty), + logicalPlan, + properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, + writeOptions = options.toMap, + orCreate = orCreate) + } + } +} + +/** + * Configuration methods common to create/replace operations and insert/overwrite operations. + * @tparam R builder type to return + */ +trait WriteConfigMethods[R] { + /** + * Add a write option. + * + * @since 3.0.0 + */ + def option(key: String, value: String): R + + /** + * Add a boolean output option. + * + * @since 3.0.0 + */ + def option(key: String, value: Boolean): R = option(key, value.toString) + + /** + * Add a long output option. + * + * @since 3.0.0 + */ + def option(key: String, value: Long): R = option(key, value.toString) + + /** + * Add a double output option. + * + * @since 3.0.0 + */ + def option(key: String, value: Double): R = option(key, value.toString) + + /** + * Add write options from a Scala Map. + * + * @since 3.0.0 + */ + def options(options: scala.collection.Map[String, String]): R + + /** + * Add write options from a Java Map. + * + * @since 3.0.0 + */ + def options(options: java.util.Map[String, String]): R +} + +/** + * Trait to restrict calls to create and replace operations. + */ +trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] { + /** + * Create a new table from the contents of the data frame. + * + * The new table's schema, partition layout, properties, and other configuration will be + * based on the configuration set on this writer. + * + * If the output table exists, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException]]. + * + * @throws org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException + * If the table already exists + */ + @throws(classOf[TableAlreadyExistsException]) + def create(): Unit + + /** + * Replace an existing table with the contents of the data frame. + * + * The existing table's schema, partition layout, properties, and other configuration will be + * replaced with the contents of the data frame and the configuration set on this writer. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException]]. + * + * @throws org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException + * If the table already exists + */ + @throws(classOf[CannotReplaceMissingTableException]) + def replace(): Unit + + /** + * Create a new table or replace an existing table with the contents of the data frame. + * + * The output table's schema, partition layout, properties, and other configuration will be based + * on the contents of the data frame and the configuration set on this writer. If the table + * exists, its configuration and data will be replaced. + */ + def createOrReplace(): Unit + + /** + * Partition the output table created by `create`, `createOrReplace`, or `replace` using + * the given columns or transforms. + * + * When specified, the table data will be stored by these values for efficient reads. + * + * For example, when a table is partitioned by day, it may be stored in a directory layout like: + *
      + *
    • `table/day=2019-06-01/`
    • + *
    • `table/day=2019-06-02/`
    • + *
    + * + * Partitioning is one of the most widely used techniques to optimize physical data layout. + * It provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number + * of distinct values in each column should typically be less than tens of thousands. + * + * @since 3.0.0 + */ + def partitionedBy(column: Column, columns: Column*): CreateTableWriter[T] + + /** + * Specifies a provider for the underlying output data source. Spark's default catalog supports + * "parquet", "json", etc. + * + * @since 3.0.0 + */ + def using(provider: String): CreateTableWriter[T] + + /** + * Add a table property. + */ + def tableProperty(property: String, value: String): CreateTableWriter[T] +} 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 88fe7a3f380ab..a2f5f03d797a2 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 @@ -3120,6 +3120,34 @@ class Dataset[T] private[sql]( new DataFrameWriter[T](this) } + /** + * Create a write configuration builder for v2 sources. + * + * This builder is used to configure and execute write operations. For example, to append to an + * existing table, run: + * + * {{{ + * df.writeTo("catalog.db.table").append() + * }}} + * + * This can also be used to create or replace existing tables: + * + * {{{ + * df.writeTo("catalog.db.table").partitionedBy($"col").createOrReplace() + * }}} + * + * @group basic + * @since 3.0.0 + */ + def writeTo(table: String): DataFrameWriterV2[T] = { + // TODO: streaming could be adapted to use this interface + if (isStreaming) { + logicalPlan.failAnalysis( + "'writeTo' can not be called on streaming Dataset/DataFrame") + } + new DataFrameWriterV2[T](table, this) + } + /** * Interface for saving the content of the streaming Dataset out into external storage. * 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 6285159dad9b0..6387333d06880 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 @@ -226,15 +226,15 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { orCreate = orCreate) :: Nil } - case AppendData(r: DataSourceV2Relation, query, _) => + case AppendData(r: DataSourceV2Relation, query, writeOptions, _) => r.table.asWritable match { case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) => - AppendDataExecV1(v1, r.options, query) :: Nil + AppendDataExecV1(v1, writeOptions.asOptions, query) :: Nil case v2 => - AppendDataExec(v2, r.options, planLater(query)) :: Nil + AppendDataExec(v2, writeOptions.asOptions, planLater(query)) :: Nil } - case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, _) => + case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) => // fail if any filter cannot be converted. correctness depends on removing all matching data. val filters = splitConjunctivePredicates(deleteExpr).map { filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse( @@ -242,13 +242,14 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { }.toArray r.table.asWritable match { case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) => - OverwriteByExpressionExecV1(v1, filters, r.options, query) :: Nil + OverwriteByExpressionExecV1(v1, filters, writeOptions.asOptions, query) :: Nil case v2 => - OverwriteByExpressionExec(v2, filters, r.options, planLater(query)) :: Nil + OverwriteByExpressionExec(v2, filters, writeOptions.asOptions, planLater(query)) :: Nil } - case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _) => - OverwritePartitionsDynamicExec(r.table.asWritable, r.options, planLater(query)) :: Nil + case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _) => + OverwritePartitionsDynamicExec( + r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil case DeleteFromTable(r: DataSourceV2Relation, condition) => if (condition.exists(SubqueryExpression.hasSubquery)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index 51993b1e9ad60..45ca3dfb9cb93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala @@ -44,14 +44,14 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { // TODO: check STREAMING_WRITE capability. It's not doable now because we don't have a // a logical plan for streaming write. - case AppendData(r: DataSourceV2Relation, _, _) if !r.table.supports(BATCH_WRITE) => + case AppendData(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) => failAnalysis(s"Table ${r.table.name()} does not support append in batch mode.") - case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _) + case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) || !r.table.supports(OVERWRITE_DYNAMIC) => failAnalysis(s"Table ${r.table.name()} does not support dynamic overwrite in batch mode.") - case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _) => + case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _, _) => expr match { case Literal(true, BooleanType) => if (!r.table.supports(BATCH_WRITE) || 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 84e0eaff2d428..a141b5e6423aa 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 @@ -69,6 +69,7 @@ import org.apache.spark.util.Utils * @groupname window_funcs Window functions * @groupname string_funcs String functions * @groupname collection_funcs Collection functions + * @groupname partition_transforms Partition transform functions * @groupname Ungrouped Support functions for DataFrames * @since 1.3.0 */ @@ -3942,6 +3943,63 @@ object functions { */ def to_csv(e: Column): Column = to_csv(e, Map.empty[String, String].asJava) + /** + * A transform for timestamps and dates to partition data into years. + * + * @group partition_transforms + * @since 3.0.0 + */ + def years(e: Column): Column = withExpr { Years(e.expr) } + + /** + * A transform for timestamps and dates to partition data into months. + * + * @group partition_transforms + * @since 3.0.0 + */ + def months(e: Column): Column = withExpr { Months(e.expr) } + + /** + * A transform for timestamps and dates to partition data into days. + * + * @group partition_transforms + * @since 3.0.0 + */ + def days(e: Column): Column = withExpr { Days(e.expr) } + + /** + * A transform for timestamps to partition data into hours. + * + * @group partition_transforms + * @since 3.0.0 + */ + def hours(e: Column): Column = withExpr { Hours(e.expr) } + + /** + * A transform for any type that partitions by a hash of the input column. + * + * @group partition_transforms + * @since 3.0.0 + */ + def bucket(numBuckets: Column, e: Column): Column = withExpr { + numBuckets.expr match { + case lit @ Literal(_, IntegerType) => + Bucket(lit, e.expr) + case _ => + throw new AnalysisException(s"Invalid number of buckets: bucket($numBuckets, $e)") + } + } + + /** + * A transform for any type that partitions by a hash of the input column. + * + * @group partition_transforms + * @since 3.0.0 + */ + def bucket(numBuckets: Int, e: Column): Column = withExpr { + Bucket(Literal(numBuckets), e.expr) + } + // scalastyle:off line.size.limit // scalastyle:off parameter.number 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 new file mode 100644 index 0000000000000..72f2e8f185637 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.InMemoryTableCatalog; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.spark.sql.functions.*; + +public class JavaDataFrameWriterV2Suite { + private static StructType schema = new StructType().add("s", "string"); + private SparkSession spark = null; + + public Dataset df() { + return spark.read().schema(schema).text(); + } + + @Before + public void createTestTable() { + this.spark = new TestSparkSession(); + spark.conf().set("spark.sql.catalog.testcat", InMemoryTableCatalog.class.getName()); + spark.sql("CREATE TABLE testcat.t (s string) USING foo"); + } + + @After + public void dropTestTable() { + spark.sql("DROP TABLE testcat.t"); + spark.stop(); + } + + @Test + 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 + public void testCreateAPI() throws TableAlreadyExistsException { + df().writeTo("testcat.t2").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").option("property", "value").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").tableProperty("property", "value").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").using("v2format").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").partitionedBy(col("s")).create(); + spark.sql("DROP TABLE testcat.t2"); + } + + @Test + public void testReplaceAPI() throws CannotReplaceMissingTableException { + df().writeTo("testcat.t").replace(); + df().writeTo("testcat.t").option("property", "value").replace(); + df().writeTo("testcat.t").tableProperty("property", "value").replace(); + df().writeTo("testcat.t").using("v2format").replace(); + df().writeTo("testcat.t").partitionedBy(col("s")).replace(); + } + + @Test + public void testCreateOrReplaceAPI() { + df().writeTo("testcat.t").createOrReplace(); + df().writeTo("testcat.t").option("property", "value").createOrReplace(); + df().writeTo("testcat.t").tableProperty("property", "value").createOrReplace(); + df().writeTo("testcat.t").using("v2format").createOrReplace(); + df().writeTo("testcat.t").partitionedBy(col("s")).createOrReplace(); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala new file mode 100644 index 0000000000000..ce0a5f21fd7ec --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -0,0 +1,507 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} + +class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with BeforeAndAfter { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.functions._ + import testImplicits._ + + private def catalog(name: String): TableCatalog = { + spark.sessionState.catalogManager.catalog(name).asTableCatalog + } + + before { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + + val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + df.createOrReplaceTempView("source") + val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data") + df2.createOrReplaceTempView("source2") + } + + after { + spark.sessionState.catalogManager.reset() + spark.sessionState.conf.clear() + } + + test("Append: basic append") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"), Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("Append: by name not position") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + val exc = intercept[AnalysisException] { + spark.table("source").withColumnRenamed("data", "d").writeTo("testcat.table_name").append() + } + + assert(exc.getMessage.contains("Cannot find data for output column")) + assert(exc.getMessage.contains("'data'")) + + checkAnswer( + spark.table("testcat.table_name"), + Seq()) + } + + test("Append: fail if table does not exist") { + val exc = intercept[NoSuchTableException] { + spark.table("source").writeTo("testcat.table_name").append() + } + + assert(exc.getMessage.contains("table_name")) + } + + test("Overwrite: overwrite by expression: true") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").overwrite(lit(true)) + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("Overwrite: overwrite by expression: id = 3") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").overwrite($"id" === 3) + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("Overwrite: by name not position") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + val exc = intercept[AnalysisException] { + spark.table("source").withColumnRenamed("data", "d") + .writeTo("testcat.table_name").overwrite(lit(true)) + } + + assert(exc.getMessage.contains("Cannot find data for output column")) + assert(exc.getMessage.contains("'data'")) + + checkAnswer( + spark.table("testcat.table_name"), + Seq()) + } + + test("Overwrite: fail if table does not exist") { + val exc = intercept[NoSuchTableException] { + spark.table("source").writeTo("testcat.table_name").overwrite(lit(true)) + } + + assert(exc.getMessage.contains("table_name")) + } + + test("OverwritePartitions: overwrite conflicting partitions") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").withColumn("id", $"id" - 2) + .writeTo("testcat.table_name").overwritePartitions() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "d"), Row(3L, "e"), Row(4L, "f"))) + } + + test("OverwritePartitions: overwrite all rows if not partitioned") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").overwritePartitions() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("OverwritePartitions: by name not position") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + val exc = intercept[AnalysisException] { + spark.table("source").withColumnRenamed("data", "d") + .writeTo("testcat.table_name").overwritePartitions() + } + + assert(exc.getMessage.contains("Cannot find data for output column")) + assert(exc.getMessage.contains("'data'")) + + checkAnswer( + spark.table("testcat.table_name"), + Seq()) + } + + test("OverwritePartitions: fail if table does not exist") { + val exc = intercept[NoSuchTableException] { + spark.table("source").writeTo("testcat.table_name").overwritePartitions() + } + + assert(exc.getMessage.contains("table_name")) + } + + test("Create: basic behavior") { + spark.table("source").writeTo("testcat.table_name").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties.isEmpty) + } + + test("Create: with using") { + spark.table("source").writeTo("testcat.table_name").using("foo").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties === Map("provider" -> "foo").asJava) + } + + test("Create: with property") { + spark.table("source").writeTo("testcat.table_name").tableProperty("prop", "value").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties === Map("prop" -> "value").asJava) + } + + test("Create: identity partitioned table") { + spark.table("source").writeTo("testcat.table_name").partitionedBy($"id").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties.isEmpty) + } + + test("Create: partitioned by years(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(years($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(YearsTransform(FieldReference("ts")))) + } + + test("Create: partitioned by months(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(months($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(MonthsTransform(FieldReference("ts")))) + } + + test("Create: partitioned by days(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(days($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(DaysTransform(FieldReference("ts")))) + } + + test("Create: partitioned by hours(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(hours($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(HoursTransform(FieldReference("ts")))) + } + + test("Create: partitioned by bucket(4, id)") { + spark.table("source") + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(bucket(4, $"id")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === + Seq(BucketTransform(LiteralValue(4, IntegerType), Seq(FieldReference("id"))))) + } + + test("Create: fail if table already exists") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + val exc = intercept[TableAlreadyExistsException] { + spark.table("source").writeTo("testcat.table_name").create() + } + + assert(exc.getMessage.contains("table_name")) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // table should not have been changed + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties === Map("provider" -> "foo").asJava) + } + + test("Replace: basic behavior") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + spark.sql("INSERT INTO TABLE testcat.table_name SELECT * FROM source") + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the initial table + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties === Map("provider" -> "foo").asJava) + + spark.table("source2") + .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) + .writeTo("testcat.table_name").replace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d", "even"), Row(5L, "e", "odd"), Row(6L, "f", "even"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType() + .add("id", LongType) + .add("data", StringType) + .add("even_or_odd", StringType)) + assert(replaced.partitioning.isEmpty) + assert(replaced.properties.isEmpty) + } + + test("Replace: partitioned table") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + spark.sql("INSERT INTO TABLE testcat.table_name SELECT * FROM source") + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the initial table + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties === Map("provider" -> "foo").asJava) + + spark.table("source2") + .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) + .writeTo("testcat.table_name").partitionedBy($"id").replace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d", "even"), Row(5L, "e", "odd"), Row(6L, "f", "even"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType() + .add("id", LongType) + .add("data", StringType) + .add("even_or_odd", StringType)) + assert(replaced.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(replaced.properties.isEmpty) + } + + test("Replace: fail if table does not exist") { + val exc = intercept[CannotReplaceMissingTableException] { + spark.table("source").writeTo("testcat.table_name").replace() + } + + assert(exc.getMessage.contains("table_name")) + } + + test("CreateOrReplace: table does not exist") { + spark.table("source2").writeTo("testcat.table_name").createOrReplace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(replaced.partitioning.isEmpty) + assert(replaced.properties.isEmpty) + } + + test("CreateOrReplace: table exists") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + spark.sql("INSERT INTO TABLE testcat.table_name SELECT * FROM source") + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the initial table + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties === Map("provider" -> "foo").asJava) + + spark.table("source2") + .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) + .writeTo("testcat.table_name").createOrReplace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d", "even"), Row(5L, "e", "odd"), Row(6L, "f", "even"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType() + .add("id", LongType) + .add("data", StringType) + .add("even_or_odd", StringType)) + assert(replaced.partitioning.isEmpty) + assert(replaced.properties.isEmpty) + } +} From 5b478416f8e3fe2f015af1b6c8faa7fe9f15c05d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 Sep 2019 15:33:12 -0700 Subject: [PATCH 55/87] [SPARK-28208][SQL][FOLLOWUP] Use `tryWithResource` pattern ### What changes were proposed in this pull request? This PR aims to use `tryWithResource` for ORC file. ### Why are the changes needed? This is a follow-up to address https://github.com/apache/spark/pull/25006#discussion_r298788206 . ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #25842 from dongjoon-hyun/SPARK-28208. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../datasources/orc/OrcFileFormat.scala | 13 ++++--- .../execution/datasources/orc/OrcUtils.scala | 8 ++--- .../v2/orc/OrcPartitionReaderFactory.scala | 22 ++++++------ .../datasources/orc/OrcQuerySuite.scala | 36 +++++++++---------- .../datasources/orc/OrcSourceSuite.scala | 8 ++--- .../sql/hive/execution/HiveDDLSuite.scala | 10 +++--- 6 files changed, 48 insertions(+), 49 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 59ee63aeab269..12c81a1dcab56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -38,10 +38,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.{SerializableConfiguration, Utils} private[sql] object OrcFileFormat { private def checkFieldName(name: String): Unit = { @@ -180,11 +179,11 @@ class OrcFileFormat val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, requiredSchema, reader, conf) - reader.close() + val requestedColIdsOrEmptyFile = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => + OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, requiredSchema, reader, conf) + } if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index f3c92f32a3694..eea9b2a8f9613 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.execution.datasources.SchemaMergeUtils import org.apache.spark.sql.types._ -import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} +import org.apache.spark.util.{ThreadUtils, Utils} object OrcUtils extends Logging { @@ -62,9 +62,9 @@ object OrcUtils extends Logging { val fs = file.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) try { - val reader = OrcFile.createReader(file, readerOptions) - val schema = reader.getSchema - reader.close() + val schema = Utils.tryWithResource(OrcFile.createReader(file, readerOptions)) { reader => + reader.getSchema + } if (schema.getFieldNames.size == 0) { None } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 7780ce45e7c69..03d58fdcb7f67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.{SerializableConfiguration, Utils} /** * A factory used to create Orc readers. @@ -74,11 +74,11 @@ case class OrcPartitionReaderFactory( val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, readDataSchema, reader, conf) - reader.close() + val requestedColIdsOrEmptyFile = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => + OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readDataSchema, reader, conf) + } if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader[InternalRow] @@ -120,11 +120,11 @@ case class OrcPartitionReaderFactory( val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, readDataSchema, reader, conf) - reader.close() + val requestedColIdsOrEmptyFile = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => + OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readDataSchema, reader, conf) + } if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index c334d0116ebbb..b8bf4b16fe53c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -194,9 +194,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("ZLIB" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("ZLIB" === reader.getCompressionKind.name) + } } // `compression` overrides `orc.compress`. @@ -211,9 +211,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("ZLIB" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("ZLIB" === reader.getCompressionKind.name) + } } } @@ -229,9 +229,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("ZLIB" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("ZLIB" === reader.getCompressionKind.name) + } } withTempPath { file => @@ -244,9 +244,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("SNAPPY" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("SNAPPY" === reader.getCompressionKind.name) + } } withTempPath { file => @@ -259,9 +259,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("NONE" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("NONE" === reader.getCompressionKind.name) + } } } @@ -645,9 +645,9 @@ class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("LZO" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("LZO" === reader.getCompressionKind.name) + } } } 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 70d15bb195d53..a2d96dd0468b4 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 @@ -372,10 +372,10 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { val orcFilePath = new Path(partFiles.head.getAbsolutePath) val readerOptions = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, readerOptions) - val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString - reader.close() - assert(version === SPARK_VERSION_SHORT) + Utils.tryWithResource(OrcFile.createReader(orcFilePath, readerOptions)) { reader => + val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString + assert(version === SPARK_VERSION_SHORT) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 487e888d588de..4253fe2e1edcb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2366,11 +2366,11 @@ class HiveDDLSuite checkAnswer(spark.table("t"), Row(1)) val maybeFile = path.listFiles().find(_.getName.startsWith("part")) - val reader = getReader(maybeFile.head.getCanonicalPath) - assert(reader.getCompressionKind.name === "ZLIB") - assert(reader.getCompressionSize == 1001) - assert(reader.getRowIndexStride == 2002) - reader.close() + Utils.tryWithResource(getReader(maybeFile.head.getCanonicalPath)) { reader => + assert(reader.getCompressionKind.name === "ZLIB") + assert(reader.getCompressionSize == 1001) + assert(reader.getRowIndexStride == 2002) + } } } } From 76ebf2241a3f2149de13d6c89adcb86325b06004 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 Sep 2019 17:54:42 -0700 Subject: [PATCH 56/87] Revert "[SPARK-29082][CORE] Skip delegation token generation if no credentials are available" This reverts commit f32f16fd68f51a94d6adb2d01a9b2e557885e656. --- .../HadoopDelegationTokenManager.scala | 22 ++++------- .../CoarseGrainedSchedulerBackend.scala | 10 ++--- .../HadoopDelegationTokenManagerSuite.scala | 39 +------------------ 3 files changed, 12 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index f769ce468e49c..759d857d56e0e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -140,21 +140,13 @@ private[spark] class HadoopDelegationTokenManager( * @param creds Credentials object where to store the delegation tokens. */ def obtainDelegationTokens(creds: Credentials): Unit = { - val currentUser = UserGroupInformation.getCurrentUser() - val hasKerberosCreds = principal != null || - Option(currentUser.getRealUser()).getOrElse(currentUser).hasKerberosCredentials() - - // Delegation tokens can only be obtained if the real user has Kerberos credentials, so - // skip creation when those are not available. - if (hasKerberosCreds) { - val freshUGI = doLogin() - freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { - override def run(): Unit = { - val (newTokens, _) = obtainDelegationTokens() - creds.addAll(newTokens) - } - }) - } + val freshUGI = doLogin() + freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val (newTokens, _) = obtainDelegationTokens() + creds.addAll(newTokens) + } + }) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index c9b408bed1163..d81070c362ba6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -427,14 +427,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val ugi = UserGroupInformation.getCurrentUser() val tokens = if (dtm.renewalEnabled) { dtm.start() - } else { + } else if (ugi.hasKerberosCredentials() || SparkHadoopUtil.get.isProxyUser(ugi)) { val creds = ugi.getCredentials() dtm.obtainDelegationTokens(creds) - if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { - SparkHadoopUtil.get.serialize(creds) - } else { - null - } + SparkHadoopUtil.get.serialize(creds) + } else { + null } if (tokens != null) { updateDelegationTokens(tokens) diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index bf53386da6304..70174f7ff939a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,14 +17,10 @@ package org.apache.spark.deploy.security -import java.security.PrivilegedExceptionAction - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION -import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.Credentials import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.security.HadoopDelegationTokenProvider private class ExceptionThrowingDelegationTokenProvider extends HadoopDelegationTokenProvider { @@ -73,37 +69,4 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(!manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) } - - test("SPARK-29082: do not fail if current user does not have credentials") { - // SparkHadoopUtil overrides the UGI configuration during initialization. That normally - // happens early in the Spark application, but here it may affect the test depending on - // how it's run, so force its initialization. - SparkHadoopUtil.get - - val krbConf = new Configuration() - krbConf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos") - - UserGroupInformation.setConfiguration(krbConf) - try { - val manager = new HadoopDelegationTokenManager(new SparkConf(false), krbConf, null) - val testImpl = new PrivilegedExceptionAction[Unit] { - override def run(): Unit = { - assert(UserGroupInformation.isSecurityEnabled()) - val creds = new Credentials() - manager.obtainDelegationTokens(creds) - assert(creds.numberOfTokens() === 0) - assert(creds.numberOfSecretKeys() === 0) - } - } - - val realUser = UserGroupInformation.createUserForTesting("realUser", Array.empty) - realUser.doAs(testImpl) - - val proxyUser = UserGroupInformation.createProxyUserForTesting("proxyUser", realUser, - Array.empty) - proxyUser.doAs(testImpl) - } finally { - UserGroupInformation.reset() - } - } } From bd05339171db00c2f2dd89702f9500ed6e1e321a Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 20 Sep 2019 14:39:24 +0900 Subject: [PATCH 57/87] [SPARK-29158][SQL] Expose SerializableConfiguration for DataSource V2 developers ### What changes were proposed in this pull request? Currently the SerializableConfiguration, which makes the Hadoop configuration serializable is private. This makes it public, with a developer annotation. ### Why are the changes needed? Many data source depend on the Hadoop configuration which may have specific components on the driver. Inside of Spark's own DataSourceV2 implementations this is frequently used (Parquet, Json, Orc, etc.) ### Does this PR introduce any user-facing change? This provides a new developer API. ### How was this patch tested? No new tests are added as this only exposes a previously developed & thoroughly used + tested component. Closes #25838 from holdenk/SPARK-29158-expose-serializableconfiguration-for-dsv2. Authored-by: Holden Karau Signed-off-by: HyukjinKwon --- .../util/SerializableConfigurationSuite.java | 28 +++++++++++++++++++ .../util/SerializableConfiguration.scala | 8 +++++- 2 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java diff --git a/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java b/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java new file mode 100644 index 0000000000000..1352653028271 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util; + +/** + * This test ensures that the API we've exposed for SerializableConfiguration is usable + * from Java. It does not test any of the serialization it's self. + */ +class SerializableConfigurationSuite { + public SerializableConfiguration compileTest() { + SerializableConfiguration scs = new SerializableConfiguration(null); + return scs; + } +} diff --git a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala index 3354a923273ff..52b309abd77f2 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala @@ -20,7 +20,13 @@ import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.hadoop.conf.Configuration -private[spark] +import org.apache.spark.annotation.{DeveloperApi, Unstable} + +/** + * Helper wrapper to serialize a Hadoop configuration. Intended for use when implementing + * DataSourceV2 readers & writers which depend on the Hadoop configuration from the driver node. + */ +@DeveloperApi @Unstable class SerializableConfiguration(@transient var value: Configuration) extends Serializable { private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.defaultWriteObject() From 5e92301723464d0876b5a7eec59c15fed0c5b98c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 19 Sep 2019 23:11:54 -0700 Subject: [PATCH 58/87] [SPARK-29161][CORE][SQL][STREAMING] Unify default wait time for waitUntilEmpty ### What changes were proposed in this pull request? This is a follow-up of the [review comment](https://github.com/apache/spark/pull/25706#discussion_r321923311). This patch unifies the default wait time to be 10 seconds as it would fit most of UTs (as they have smaller timeouts) and doesn't bring additional latency since it will return if the condition is met. This patch doesn't touch the one which waits 100000 milliseconds (100 seconds), to not break anything unintentionally, though I'd rather questionable that we really need to wait for 100 seconds. ### Why are the changes needed? It simplifies the test code and get rid of various heuristic values on timeout. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? CI build will test the patch, as it would be the best environment to test the patch (builds are running there). Closes #25837 from HeartSaVioR/MINOR-unify-default-wait-time-for-wait-until-empty. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 11 +++++++ .../org/apache/spark/AccumulatorSuite.scala | 2 +- .../ExecutorAllocationManagerSuite.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 2 +- .../apache/spark/SparkContextInfoSuite.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 2 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 7 ++--- .../metrics/InputOutputMetricsSuite.scala | 8 ++--- .../spark/scheduler/DAGSchedulerSuite.scala | 21 ++++++------- .../spark/scheduler/SparkListenerSuite.scala | 31 +++++++++---------- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +-- .../apache/spark/sql/SessionStateSuite.scala | 6 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 6 ++-- .../FileDataSourceV2FallBackSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 4 +-- .../StreamingQueryListenerSuite.scala | 6 ++-- .../StreamingQueryListenersConfSuite.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 10 +++--- .../sql/util/DataFrameCallbackSuite.scala | 18 +++++------ .../util/ExecutionListenerManagerSuite.scala | 4 +-- .../streaming/StreamingListenerSuite.scala | 2 +- 26 files changed, 82 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 41ae3ae3b758a..9aeb128fcc491 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -264,7 +264,7 @@ private[spark] object TestUtils { try { body(listener) } finally { - sc.listenerBus.waitUntilEmpty(TimeUnit.SECONDS.toMillis(10)) + sc.listenerBus.waitUntilEmpty() sc.listenerBus.removeListener(listener) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 302ebd30da228..bbbddd86cad39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -186,6 +186,17 @@ private[spark] class LiveListenerBus(conf: SparkConf) { metricsSystem.registerSource(metrics) } + /** + * For testing only. Wait until there are no more events in the queue, or until the default + * wait time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue + * emptied. + * Exposed for testing. + */ + @throws(classOf[TimeoutException]) + private[spark] def waitUntilEmpty(): Unit = { + waitUntilEmpty(TimeUnit.SECONDS.toMillis(10)) + } + /** * For testing only. Wait until there are no more events in the queue, or until the specified * time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 435665d8a1ce2..a75cf3f0381df 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -126,7 +126,7 @@ private[spark] object AccumulatorSuite { sc.addSparkListener(listener) testBody // wait until all events have been processed before proceeding to assert things - sc.listenerBus.waitUntilEmpty(10 * 1000) + sc.listenerBus.waitUntilEmpty() val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) val isSet = accums.exists { a => a.name == Some(PEAK_EXECUTION_MEMORY) && a.value.exists(_.asInstanceOf[Long] > 0L) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 07fb323cfc355..460714f204a3a 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -64,7 +64,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private def post(event: SparkListenerEvent): Unit = { listenerBus.post(event) - listenerBus.waitUntilEmpty(1000) + listenerBus.waitUntilEmpty() } test("initialize dynamic allocation in SparkContext") { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 923c9c90447fd..4c9d7e9faeb2a 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -498,7 +498,7 @@ object ShuffleSuite { job - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() AggregatedShuffleMetrics(recordsWritten, recordsRead, bytesWritten, bytesRead) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 536b4aec75623..09510edcb4499 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -63,7 +63,7 @@ class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(sc.getRDDStorageInfo.length === 0) rdd.collect() - sc.listenerBus.waitUntilEmpty(10000) + sc.listenerBus.waitUntilEmpty() eventually(timeout(10.seconds), interval(100.milliseconds)) { assert(sc.getRDDStorageInfo.length === 1) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 786f55c96a3e8..eca49e5efdbbb 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -716,7 +716,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } x }.collect() - sc.listenerBus.waitUntilEmpty(10000) + sc.listenerBus.waitUntilEmpty() // As executors will send the metrics of running tasks via heartbeat, we can use this to check // whether there is any running task. eventually(timeout(10.seconds)) { diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index cbdf1755b0c5b..d953211c13b1d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -29,9 +29,6 @@ import org.apache.spark.util.SparkConfWithEnv class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { - /** Length of time to wait while draining listener events. */ - private val WAIT_TIMEOUT_MILLIS = 10000 - test("verify that correct log urls get propagated from workers") { sc = new SparkContext("local-cluster[2,1,1024]", "test") @@ -41,7 +38,7 @@ class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) // Browse to each URL to check that it's valid @@ -61,7 +58,7 @@ class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] assert(listeners.size === 1) val listener = listeners(0) diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index c7bd0c905d027..f9511045ee89e 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -182,7 +182,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext .reduceByKey(_ + _) .saveAsTextFile(tmpFile.toURI.toString) - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() assert(inputRead == numRecords) assert(outputWritten == numBuckets) @@ -243,7 +243,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext val taskMetrics = new ArrayBuffer[Long]() // Avoid receiving earlier taskEnd events - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { @@ -253,7 +253,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext job - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() taskMetrics.sum } @@ -293,7 +293,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext try { rdd.saveAsTextFile(outPath.toString) - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() assert(taskBytesWritten.length == 2) val outFiles = fs.listStatus(outPath).filter(_.getPath.getName != "_SUCCESS") taskBytesWritten.zip(outFiles).foreach { case (bytes, fileStatus) => diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7cb7eceec615b..b86d24deb15fc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -172,9 +172,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def applicationAttemptId(): Option[String] = None } - /** Length of time to wait while draining listener events. */ - val WAIT_TIMEOUT_MILLIS = 10000 - /** * Listeners which records some information to verify in UTs. Getter-kind methods in this class * ensures the value is returned after ensuring there's no event to process, as well as the @@ -230,7 +227,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi _endedTasks.toSet } - private def waitForListeners(): Unit = sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + private def waitForListeners(): Unit = sc.listenerBus.waitUntilEmpty() } var sparkListener: EventInfoRecordingListener = null @@ -839,7 +836,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val testRdd = new MyRDD(sc, 0, Nil) val waiter = scheduler.submitJob(testRdd, func, Seq.empty, CallSite.empty, resultHandler, properties) - sc.listenerBus.waitUntilEmpty(1000L) + sc.listenerBus.waitUntilEmpty() assert(assertionError.get() === null) } @@ -957,7 +954,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi completeNextResultStageWithSuccess(1, 1) // Confirm job finished successfully - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(ended) assert(results === (0 until parts).map { idx => idx -> 42 }.toMap) assertDataStructuresEmpty() @@ -994,7 +991,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } else { // Stage should have been aborted and removed from running stages assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(ended) jobResult match { case JobFailed(reason) => @@ -1116,7 +1113,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi completeNextResultStageWithSuccess(2, 1) assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(ended) assert(results === Map(0 -> 42)) } @@ -1175,7 +1172,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the result stage. completeNextResultStageWithSuccess(1, 1) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() assertDataStructuresEmpty() } @@ -1204,7 +1201,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the result stage. completeNextResultStageWithSuccess(1, 0) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() assertDataStructuresEmpty() } @@ -1230,7 +1227,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi null)) // Assert the stage has been cancelled. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() assert(failure.getMessage.startsWith("Job aborted due to stage failure: Could not recover " + "from a failed barrier ResultStage.")) } @@ -2668,7 +2665,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi sc.parallelize(1 to tasks, tasks).foreach { _ => accum.add(1L) } - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(foundCount.get() === tasks) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 8903e1054f53d..cda6fbad5419f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -38,9 +38,6 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match import LiveListenerBus._ - /** Length of time to wait while draining listener events. */ - val WAIT_TIMEOUT_MILLIS = 10000 - val jobCompletionTime = 1421191296660L private val mockSparkContext: SparkContext = Mockito.mock(classOf[SparkContext]) @@ -65,7 +62,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match sc.listenerBus.addToSharedQueue(listener) sc.listenerBus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() sc.stop() assert(listener.sparkExSeen) @@ -97,7 +94,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // Starting listener bus should flush all buffered events bus.start(mockSparkContext, mockMetricsSystem) Mockito.verify(mockMetricsSystem).registerSource(bus.metrics) - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() assert(counter.count === 5) assert(sharedQueueSize(bus) === 0) assert(eventProcessingTimeCount(bus) === 5) @@ -223,7 +220,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match rdd2.setName("Target RDD") rdd2.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val (stageInfo, taskInfoMetrics) = listener.stageInfos.head @@ -248,7 +245,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match rdd3.setName("Trois") rdd1.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val stageInfo1 = listener.stageInfos.keys.find(_.stageId == 0).get stageInfo1.rddInfos.size should be {1} // ParallelCollectionRDD @@ -257,7 +254,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match listener.stageInfos.clear() rdd2.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get stageInfo2.rddInfos.size should be {3} @@ -266,7 +263,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match listener.stageInfos.clear() rdd3.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {2} // Shuffle map stage + result stage val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 3).get stageInfo3.rddInfos.size should be {1} // ShuffledRDD @@ -282,7 +279,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val rdd2 = rdd1.map(_.toString) sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1)) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val (stageInfo, _) = listener.stageInfos.head @@ -310,7 +307,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val numSlices = 16 val d = sc.parallelize(0 to 10000, numSlices).map(w) d.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be (1) val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") @@ -321,7 +318,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match d4.setName("A Cogroup") d4.collectAsMap() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be (4) listener.stageInfos.foreach { case (stageInfo, taskInfoMetrics) => /** @@ -372,7 +369,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match .reduce { case (x, y) => x } assert(result === 1.to(maxRpcMessageSize).toArray) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.contains(TASK_INDEX)) @@ -388,7 +385,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.isEmpty) @@ -443,7 +440,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() // The exception should be caught, and the event should be propagated to other listeners assert(jobCounter1.count === 5) @@ -513,7 +510,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // after we post one event, both interrupting listeners should get removed, and the // event log queue should be removed bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() assert(bus.activeQueues() === Set(SHARED_QUEUE, APP_STATUS_QUEUE)) assert(bus.findListenersByClass[BasicJobCounter]().size === 2) assert(bus.findListenersByClass[InterruptingListener]().size === 0) @@ -522,7 +519,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // posting more events should be fine, they'll just get processed from the OK queue. (0 until 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() assert(counter1.count === 6) assert(counter2.count === 6) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 6e1ee6da9200d..edafdaa54945a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -849,7 +849,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi sparkContext.addSparkListener(jobListener) try { val result = f - sparkContext.listenerBus.waitUntilEmpty(10000L) + sparkContext.listenerBus.waitUntilEmpty() assert(numJobTrigered === 0) result } finally { 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 6a2eac9e7e21c..2306c3ff039f1 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 @@ -2114,7 +2114,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { val df = spark.read.json(path.getCanonicalPath) assert(df.columns === Array("i", "p")) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(numJobs == 1) } } 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 23848d90dc53d..e4c10d1990f25 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 @@ -488,7 +488,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { sparkContext.addSparkListener(bytesReadListener) try { spark.read.csv(path).limit(1).collect() - sparkContext.listenerBus.waitUntilEmpty(1000L) + sparkContext.listenerBus.waitUntilEmpty() assert(bytesReads.sum === 7860) } finally { sparkContext.removeSparkListener(bytesReadListener) 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 80c1e24bfa568..28a027690db04 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 @@ -2609,14 +2609,14 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { } // Make sure no spurious job starts are pending in the listener bus. - sparkContext.listenerBus.waitUntilEmpty(500) + sparkContext.listenerBus.waitUntilEmpty() sparkContext.addSparkListener(listener) try { // Execute the command. sql("show databases").head() // Make sure we have seen all events triggered by DataFrame.show() - sparkContext.listenerBus.waitUntilEmpty(500) + sparkContext.listenerBus.waitUntilEmpty() } finally { sparkContext.removeSparkListener(listener) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala index 1d461a03fd1f6..31957a99e15af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala @@ -158,7 +158,7 @@ class SessionStateSuite extends SparkFunSuite { assert(forkedSession ne activeSession) assert(forkedSession.listenerManager ne activeSession.listenerManager) runCollectQueryOn(forkedSession) - activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) + activeSession.sparkContext.listenerBus.waitUntilEmpty() assert(collectorA.commands.length == 1) // forked should callback to A assert(collectorA.commands(0) == "collect") @@ -166,14 +166,14 @@ class SessionStateSuite extends SparkFunSuite { // => changes to forked do not affect original forkedSession.listenerManager.register(collectorB) runCollectQueryOn(activeSession) - activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) + activeSession.sparkContext.listenerBus.waitUntilEmpty() assert(collectorB.commands.isEmpty) // original should not callback to B assert(collectorA.commands.length == 2) // original should still callback to A assert(collectorA.commands(1) == "collect") // <= changes to original do not affect forked activeSession.listenerManager.register(collectorC) runCollectQueryOn(forkedSession) - activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) + activeSession.sparkContext.listenerBus.waitUntilEmpty() assert(collectorC.commands.isEmpty) // forked should not callback to C assert(collectorA.commands.length == 3) // forked should still callback to A assert(collectorB.commands.length == 1) // forked should still callback to B diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 2a034bcdc3f00..268fb65144e72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -360,13 +360,13 @@ class UDFSuite extends QueryTest with SharedSparkSession { .withColumn("b", udf1($"a", lit(10))) df.cache() df.write.saveAsTable("t") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(numTotalCachedHit == 1, "expected to be cached in saveAsTable") df.write.insertInto("t") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(numTotalCachedHit == 2, "expected to be cached in insertInto") df.write.save(path.getCanonicalPath) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(numTotalCachedHit == 3, "expected to be cached in save for native") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index cd804adfa2133..2b3340527a4e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -173,7 +173,7 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { withTempPath { path => val inputData = spark.range(10) inputData.write.format(format).save(path.getCanonicalPath) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 1) assert(commands.head._1 == "save") assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index e7f1c42d7d7c5..ccefb53fc4b39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -388,7 +388,7 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { // Assume the execution plan is // PhysicalRDD(nodeId = 0) data.write.format("json").save(file.getAbsolutePath) - sparkContext.listenerBus.waitUntilEmpty(10000) + sparkContext.listenerBus.waitUntilEmpty() val executionIds = currentExecutionIds().diff(previousExecutionIds) assert(executionIds.size === 1) val executionId = executionIds.head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 90966d2efec23..88864ccec7523 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -480,7 +480,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils // At the beginning of this test case, there should be no live data in the listener. assert(listener.noLiveData()) spark.sparkContext.parallelize(1 to 10).foreach(i => ()) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() // Listener should ignore the non-SQL stages, as the stage data are only removed when SQL // execution ends, which will not be triggered for non-SQL jobs. assert(listener.noLiveData()) @@ -673,7 +673,7 @@ class SQLAppStatusListenerMemoryLeakSuite extends SparkFunSuite { case e: SparkException => // This is expected for a failed job } } - sc.listenerBus.waitUntilEmpty(10000) + sc.listenerBus.waitUntilEmpty() val statusStore = spark.sharedState.statusStore assert(statusStore.executionsCount() <= 50) assert(statusStore.planGraphCount() <= 50) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index d84dc62a696b9..4c58cb85c4d36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -49,7 +49,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(spark.streams.active.isEmpty) assert(spark.streams.listListeners().isEmpty) // Make sure we don't leak any events to the next test - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() } testQuietly("single listener, check trigger events are generated correctly") { @@ -320,7 +320,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { q.recentProgress.size > 1 && q.recentProgress.size <= 11 } testStream(input.toDS)(actions: _*) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() // 11 is the max value of the possible numbers of events. assert(numProgressEvent > 1 && numProgressEvent <= 11) } finally { @@ -343,7 +343,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { AddData(mem, 1, 2, 3), CheckAnswer(1, 2, 3) ) - session.sparkContext.listenerBus.waitUntilEmpty(5000) + session.sparkContext.listenerBus.waitUntilEmpty() } def assertEventsCollected(collector: EventCollector): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 7801d968e901d..d538d93b845b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -39,7 +39,7 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { StopStream ) - spark.sparkContext.listenerBus.waitUntilEmpty(5000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(TestListener.queryStartedEvent != null) assert(TestListener.queryTerminatedEvent != null) 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 b98626a34cc29..01a03e484c90d 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 @@ -277,7 +277,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with .format(classOf[NoopDataSource].getName) .mode(SaveMode.Append) .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[AppendData]) // overwrite mode creates `OverwriteByExpression` @@ -285,21 +285,21 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with .format(classOf[NoopDataSource].getName) .mode(SaveMode.Overwrite) .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[OverwriteByExpression]) // By default the save mode is `ErrorIfExists` for data source v2. spark.range(10).write .format(classOf[NoopDataSource].getName) .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[AppendData]) spark.range(10).write .format(classOf[NoopDataSource].getName) .mode("default") .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[AppendData]) } finally { spark.listenerManager.unregister(listener) @@ -1058,7 +1058,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with checkDatasetUnorderly( spark.read.parquet(dir.getCanonicalPath).as[(Long, Long)], 0L -> 0L, 1L -> 1L, 2L -> 2L) - sparkContext.listenerBus.waitUntilEmpty(10000) + sparkContext.listenerBus.waitUntilEmpty() assert(jobDescriptions.asScala.toList.exists( _.contains("Listing leaf files and directories for 3 paths"))) } finally { 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 18a82f0f84283..e96f05384c879 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 @@ -49,7 +49,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { df.select("i").collect() df.filter($"i" > 0).count() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 2) assert(metrics(0)._1 == "collect") @@ -80,7 +80,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException](df.select(errorUdf($"i")).collect()) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 1) assert(metrics(0)._1 == "collect") assert(metrics(0)._2.analyzed.isInstanceOf[Project]) @@ -110,12 +110,12 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { df.collect() // Wait for the first `collect` to be caught by our listener. Otherwise the next `collect` will // reset the plan metrics. - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() df.collect() Seq(1 -> "a", 2 -> "a").toDF("i", "j").groupBy("i").count().collect() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 3) assert(metrics(0) === 1) assert(metrics(1) === 1) @@ -163,7 +163,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { // For this simple case, the peakExecutionMemory of a stage should be the data size of the // aggregate operator, as we only have one memory consuming operator per stage. - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 2) assert(metrics(0) == topAggDataSize) assert(metrics(1) == bottomAggDataSize) @@ -187,7 +187,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { withTempPath { path => spark.range(10).write.format("json").save(path.getCanonicalPath) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 1) assert(commands.head._1 == "save") assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) @@ -198,7 +198,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { withTable("tab") { sql("CREATE TABLE tab(i long) using parquet") // adds commands(1) via onSuccess spark.range(10).write.insertInto("tab") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 3) assert(commands(2)._1 == "insertInto") assert(commands(2)._2.isInstanceOf[InsertIntoStatement]) @@ -209,7 +209,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { withTable("tab") { spark.range(10).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("tab") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 5) assert(commands(4)._1 == "saveAsTable") assert(commands(4)._2.isInstanceOf[CreateTable]) @@ -221,7 +221,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { val e = intercept[AnalysisException] { spark.range(10).select($"id", $"id").write.insertInto("tab") } - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(errors.length == 1) assert(errors.head._1 == "insertInto") assert(errors.head._2 == e) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala index 79819e7655414..2fd6cb220ea3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala @@ -34,13 +34,13 @@ class ExecutionListenerManagerSuite extends SparkFunSuite with LocalSparkSession spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() spark.sql("select 1").collect() - spark.sparkContext.listenerBus.waitUntilEmpty(1000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(INSTANCE_COUNT.get() === 1) assert(CALLBACK_COUNT.get() === 1) val cloned = spark.cloneSession() cloned.sql("select 1").collect() - spark.sparkContext.listenerBus.waitUntilEmpty(1000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(INSTANCE_COUNT.get() === 1) assert(CALLBACK_COUNT.get() === 2) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 9cd5d8c0e95e1..01dbe75a9eae8 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -227,7 +227,7 @@ class StreamingListenerSuite extends TestSuiteBase with LocalStreamingContext wi // Post a Streaming event after stopping StreamingContext val receiverInfoStopped = ReceiverInfo(0, "test", false, "localhost", "0") ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfoStopped)) - ssc.sparkContext.listenerBus.waitUntilEmpty(1000) + ssc.sparkContext.listenerBus.waitUntilEmpty() // The StreamingListener should not receive any event verifyNoMoreInteractions(streamingListener) } From ec8a1a8e8869b000a8001126ca013b753acddf6a Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 20 Sep 2019 21:41:09 +0900 Subject: [PATCH 59/87] [SPARK-29122][SQL] Propagate all the SQL conf to executors in SQLQueryTestSuite ### What changes were proposed in this pull request? This pr is to propagate all the SQL configurations to executors in `SQLQueryTestSuite`. When the propagation enabled in the tests, a potential bug below becomes apparent; ``` CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet; .... select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4): QueryOutput(select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4),struct<>,java.lang.IllegalArgumentException [info] requirement failed: MutableProjection cannot use UnsafeRow for output data types: decimal(38,0)) (SQLQueryTestSuite.scala:380) ``` The root culprit is that `InterpretedMutableProjection` has incorrect validation in the interpreter mode: `validExprs.forall { case (e, _) => UnsafeRow.isFixedLength(e.dataType) }`. This validation should be the same with the condition (`isMutable`) in `HashAggregate.supportsAggregate`: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L1126 ### Why are the changes needed? Bug fixes. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added tests in `AggregationQuerySuite` Closes #25831 from maropu/SPARK-29122. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../expressions/InterpretedMutableProjection.scala | 6 +++--- .../sql/catalyst/expressions/windowExpressions.scala | 2 +- .../org/apache/spark/sql/SQLQueryTestSuite.scala | 8 ++++---- .../sql/hive/execution/AggregationQuerySuite.scala | 12 +++++++++++- 4 files changed, 19 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala index 5c8aa4e2e9d83..a2daec0b1ade1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala @@ -50,11 +50,11 @@ class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mutable def currentValue: InternalRow = mutableRow override def target(row: InternalRow): MutableProjection = { - // If `mutableRow` is `UnsafeRow`, `MutableProjection` accepts fixed-length types only + // If `mutableRow` is `UnsafeRow`, `MutableProjection` accepts mutable types only require(!row.isInstanceOf[UnsafeRow] || - validExprs.forall { case (e, _) => UnsafeRow.isFixedLength(e.dataType) }, + validExprs.forall { case (e, _) => UnsafeRow.isMutable(e.dataType) }, "MutableProjection cannot use UnsafeRow for output data types: " + - validExprs.map(_._1.dataType).filterNot(UnsafeRow.isFixedLength) + validExprs.map(_._1.dataType).filterNot(UnsafeRow.isMutable) .map(_.catalogString).mkString(", ")) mutableRow = row this diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index 0b674d025d1ac..74df1e7b8e8c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -606,7 +606,7 @@ case class NTile(buckets: Expression) extends RowNumberLike with SizeBasedWindow zero, zero, zero, - (n / buckets).cast(IntegerType), + (n.cast(DecimalType.IntDecimal) / buckets.cast(DecimalType.IntDecimal)).cast(IntegerType), (n % buckets).cast(IntegerType) ) 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 abecaa7a9b16a..f32575794f946 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 @@ -310,9 +310,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { 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) - // Propagate the SQL conf ANSI_ENABLED to executor. - // TODO: remove this after SPARK-29122 is resolved. - localSparkSession.sparkContext.setLocalProperty(SQLConf.ANSI_ENABLED.key, "true") + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } @@ -406,7 +404,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { 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 = hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) + 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) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index d06cc1c0a88ac..8c7e5bf5ac1d4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -24,7 +24,7 @@ import test.org.apache.spark.sql.MyDoubleAvg import test.org.apache.spark.sql.MyDoubleSum import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.{CodegenObjectFactoryMode, UnsafeRow} import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -1018,6 +1018,16 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te val agg2 = agg1.groupBy($"text").agg(sum($"avg_res")) checkAnswer(agg2, Row("a", BigDecimal("11.9999999994857142860000"))) } + + test("SPARK-29122: hash-based aggregates for unfixed-length decimals in the interpreter mode") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString) { + withTempView("t") { + spark.range(3).selectExpr("CAST(id AS decimal(38, 0)) a").createOrReplaceTempView("t") + checkAnswer(sql("SELECT SUM(a) FROM t"), Row(java.math.BigDecimal.valueOf(3))) + } + } + } } From eb7ee6834dc7936b11d85daa7770c10d23842058 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 20 Sep 2019 22:04:32 +0800 Subject: [PATCH 60/87] [SPARK-29062][SQL] Add V1_BATCH_WRITE to the TableCapabilityChecks ### What changes were proposed in this pull request? Currently the checks in the Analyzer require that V2 Tables have BATCH_WRITE defined for all tables that have V1 Write fallbacks. This is confusing as these tables may not have the V2 writer interface implemented yet. This PR adds this table capability to these checks. In addition, this allows V2 tables to leverage the V1 APIs for DataFrameWriter.save if they do extend the V1_BATCH_WRITE capability. This way, these tables can continue to receive partitioning information and also perform checks for the existence of tables, and support all SaveModes. ### Why are the changes needed? Partitioned saves through DataFrame.write are otherwise broken for V2 tables that support the V1 write API. ### Does this PR introduce any user-facing change? No ### How was this patch tested? V1WriteFallbackSuite Closes #25767 from brkyvz/bwcheck. Authored-by: Burak Yavuz Signed-off-by: Wenchen Fan --- .../apache/spark/sql/DataFrameWriter.scala | 9 +- .../datasources/v2/TableCapabilityCheck.scala | 14 +- .../connector/TableCapabilityCheckSuite.scala | 28 ++-- .../sql/connector/V1WriteFallbackSuite.scala | 130 ++++++++++++++++-- 4 files changed, 153 insertions(+), 28 deletions(-) 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 289a976c6db9e..edf0963e71e81 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 @@ -253,11 +253,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val maybeV2Provider = lookupV2Provider() if (maybeV2Provider.isDefined) { - if (partitioningColumns.nonEmpty) { - throw new AnalysisException( - "Cannot write data to TableProvider implementation if partition columns are specified.") - } - val provider = maybeV2Provider.get val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, df.sparkSession.sessionState.conf) @@ -267,6 +262,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ provider.getTable(dsOptions) match { case table: SupportsWrite if table.supports(BATCH_WRITE) => + if (partitioningColumns.nonEmpty) { + throw new AnalysisException("Cannot write data to TableProvider implementation " + + "if partition columns are specified.") + } lazy val relation = DataSourceV2Relation.create(table, dsOptions) modeForDSV2 match { case SaveMode.Append => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index 45ca3dfb9cb93..509a5f7139cca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.types.BooleanType @@ -32,6 +33,10 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { private def failAnalysis(msg: String): Unit = throw new AnalysisException(msg) + private def supportsBatchWrite(table: Table): Boolean = { + table.supportsAny(BATCH_WRITE, V1_BATCH_WRITE) + } + override def apply(plan: LogicalPlan): Unit = { plan foreach { case r: DataSourceV2Relation if !r.table.supports(BATCH_READ) => @@ -43,8 +48,7 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { // TODO: check STREAMING_WRITE capability. It's not doable now because we don't have a // a logical plan for streaming write. - - case AppendData(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) => + case AppendData(r: DataSourceV2Relation, _, _, _) if !supportsBatchWrite(r.table) => failAnalysis(s"Table ${r.table.name()} does not support append in batch mode.") case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _, _) @@ -54,13 +58,13 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _, _) => expr match { case Literal(true, BooleanType) => - if (!r.table.supports(BATCH_WRITE) || - !r.table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER)) { + if (!supportsBatchWrite(r.table) || + !r.table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER)) { failAnalysis( s"Table ${r.table.name()} does not support truncate in batch mode.") } case _ => - if (!r.table.supports(BATCH_WRITE) || !r.table.supports(OVERWRITE_BY_FILTER)) { + if (!supportsBatchWrite(r.table) || !r.table.supports(OVERWRITE_BY_FILTER)) { failAnalysis(s"Table ${r.table.name()} does not support " + "overwrite by filter in batch mode.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 39f4085a9baf9..ce6d56cf84df1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -98,16 +98,19 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { } test("AppendData: check correct capabilities") { - val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(BATCH_WRITE), CaseInsensitiveStringMap.empty), - TestRelation) + Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => + val plan = AppendData.byName( + DataSourceV2Relation.create(CapabilityTable(write), CaseInsensitiveStringMap.empty), + TestRelation) - TableCapabilityCheck.apply(plan) + TableCapabilityCheck.apply(plan) + } } test("Truncate: check missing capabilities") { Seq(CapabilityTable(), CapabilityTable(BATCH_WRITE), + CapabilityTable(V1_BATCH_WRITE), CapabilityTable(TRUNCATE), CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => @@ -125,7 +128,9 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("Truncate: check correct capabilities") { Seq(CapabilityTable(BATCH_WRITE, TRUNCATE), - CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => + CapabilityTable(V1_BATCH_WRITE, TRUNCATE), + CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER), + CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, @@ -137,6 +142,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("OverwriteByExpression: check missing capabilities") { Seq(CapabilityTable(), + CapabilityTable(V1_BATCH_WRITE), CapabilityTable(BATCH_WRITE), CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => @@ -153,12 +159,14 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { } test("OverwriteByExpression: check correct capabilities") { - val table = CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER) - val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, - EqualTo(AttributeReference("x", LongType)(), Literal(5))) + Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => + val table = CapabilityTable(write, OVERWRITE_BY_FILTER) + val plan = OverwriteByExpression.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + EqualTo(AttributeReference("x", LongType)(), Literal(5))) - TableCapabilityCheck.apply(plan) + TableCapabilityCheck.apply(plan) + } } test("OverwritePartitionsDynamic: check missing capabilities") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index 7cd6ba21b56ec..de843ba4375d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -24,11 +24,12 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{DataFrame, QueryTest, Row, SaveMode, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} -import org.apache.spark.sql.sources.{DataSourceRegister, Filter, InsertableRelation} +import org.apache.spark.sql.execution.datasources.{DataSource, DataSourceUtils} +import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -52,7 +53,11 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before test("append fallback") { val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") df.write.mode("append").option("name", "t1").format(v2Format).save() + checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df) + assert(InMemoryV1Provider.tables("t1").schema === df.schema.asNullable) + assert(InMemoryV1Provider.tables("t1").partitioning.isEmpty) + df.write.mode("append").option("name", "t1").format(v2Format).save() checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df.union(df)) } @@ -65,6 +70,59 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before df2.write.mode("overwrite").option("name", "t1").format(v2Format).save() checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df2) } + + SaveMode.values().foreach { mode => + test(s"save: new table creations with partitioning for table - mode: $mode") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + df.write.mode(mode).option("name", "t1").format(format).partitionBy("a").save() + + checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df) + assert(InMemoryV1Provider.tables("t1").schema === df.schema.asNullable) + assert(InMemoryV1Provider.tables("t1").partitioning.sameElements( + Array(IdentityTransform(FieldReference(Seq("a")))))) + } + } + + test("save: default mode is ErrorIfExists") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + + df.write.option("name", "t1").format(format).partitionBy("a").save() + // default is ErrorIfExists, and since a table already exists we throw an exception + val e = intercept[AnalysisException] { + df.write.option("name", "t1").format(format).partitionBy("a").save() + } + assert(e.getMessage.contains("already exists")) + } + + test("save: Ignore mode") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + + df.write.option("name", "t1").format(format).partitionBy("a").save() + // no-op + df.write.option("name", "t1").format(format).mode("ignore").partitionBy("a").save() + + checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df) + } + + test("save: tables can perform schema and partitioning checks if they already exist") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + + df.write.option("name", "t1").format(format).partitionBy("a").save() + val e2 = intercept[IllegalArgumentException] { + df.write.mode("append").option("name", "t1").format(format).partitionBy("b").save() + } + assert(e2.getMessage.contains("partitioning")) + + val e3 = intercept[IllegalArgumentException] { + Seq((1, "x")).toDF("c", "d").write.mode("append").option("name", "t1").format(format) + .save() + } + assert(e3.getMessage.contains("schema")) + } } class V1WriteFallbackSessionCatalogSuite @@ -114,26 +172,83 @@ private object InMemoryV1Provider { } } -class InMemoryV1Provider extends TableProvider with DataSourceRegister { +class InMemoryV1Provider + extends TableProvider + with DataSourceRegister + with CreatableRelationProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { - InMemoryV1Provider.tables.getOrElseUpdate(options.get("name"), { + + InMemoryV1Provider.tables.getOrElse(options.get("name"), { new InMemoryTableWithV1Fallback( "InMemoryTableWithV1Fallback", - new StructType().add("a", IntegerType).add("b", StringType), - Array(IdentityTransform(FieldReference(Seq("a")))), + new StructType(), + Array.empty, options.asCaseSensitiveMap() ) }) } override def shortName(): String = "in-memory" + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + val _sqlContext = sqlContext + + val partitioning = parameters.get(DataSourceUtils.PARTITIONING_COLUMNS_KEY).map { value => + DataSourceUtils.decodePartitioningColumns(value).map { partitioningColumn => + IdentityTransform(FieldReference(partitioningColumn)) + } + }.getOrElse(Nil) + + val tableName = parameters("name") + val tableOpt = InMemoryV1Provider.tables.get(tableName) + val table = tableOpt.getOrElse(new InMemoryTableWithV1Fallback( + "InMemoryTableWithV1Fallback", + data.schema.asNullable, + partitioning.toArray, + Map.empty[String, String].asJava + )) + if (tableOpt.isEmpty) { + InMemoryV1Provider.tables.put(tableName, table) + } else { + if (data.schema.asNullable != table.schema) { + throw new IllegalArgumentException("Wrong schema provided") + } + if (!partitioning.sameElements(table.partitioning)) { + throw new IllegalArgumentException("Wrong partitioning provided") + } + } + + def getRelation: BaseRelation = new BaseRelation { + override def sqlContext: SQLContext = _sqlContext + override def schema: StructType = table.schema + } + + if (mode == SaveMode.ErrorIfExists && tableOpt.isDefined) { + throw new AnalysisException("Table already exists") + } else if (mode == SaveMode.Ignore && tableOpt.isDefined) { + // do nothing + return getRelation + } + val writer = table.newWriteBuilder(new CaseInsensitiveStringMap(parameters.asJava)) + if (mode == SaveMode.Overwrite) { + writer.asInstanceOf[SupportsTruncate].truncate() + } + writer.asInstanceOf[V1WriteBuilder].buildForV1Write().insert(data, overwrite = false) + getRelation + } } class InMemoryTableWithV1Fallback( override val name: String, override val schema: StructType, override val partitioning: Array[Transform], - override val properties: util.Map[String, String]) extends Table with SupportsWrite { + override val properties: util.Map[String, String]) + extends Table + with SupportsWrite { partitioning.foreach { t => if (!t.isInstanceOf[IdentityTransform]) { @@ -142,7 +257,6 @@ class InMemoryTableWithV1Fallback( } override def capabilities: util.Set[TableCapability] = Set( - TableCapability.BATCH_WRITE, TableCapability.V1_BATCH_WRITE, TableCapability.OVERWRITE_BY_FILTER, TableCapability.TRUNCATE).asJava From a23ad25ba43ec0974986b3ffdb4df671c8c8b9ae Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 20 Sep 2019 08:52:30 -0700 Subject: [PATCH 61/87] [SPARK-29158][SQL][FOLLOW-UP] Create an actual test case under `src/test` and minor documentation correction ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/25838 and proposes to create an actual test case under `src/test`. Previously, compile only test existed at `src/main`. Also, just changed the wordings in `SerializableConfiguration` just only to describe what it does (remove other words). ### Why are the changes needed? Tests codes should better exist in `src/test` not `src/main`. Also, it should better test a basic functionality. ### Does this PR introduce any user-facing change? No except minor doc change. ### How was this patch tested? Unit test was added. Closes #25867 from HyukjinKwon/SPARK-29158. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../util/SerializableConfigurationSuite.java | 28 ---------- .../util/SerializableConfiguration.scala | 5 +- .../util/SerializableConfigurationSuite.java | 55 +++++++++++++++++++ 3 files changed, 58 insertions(+), 30 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java create mode 100644 core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java diff --git a/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java b/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java deleted file mode 100644 index 1352653028271..0000000000000 --- a/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.util; - -/** - * This test ensures that the API we've exposed for SerializableConfiguration is usable - * from Java. It does not test any of the serialization it's self. - */ -class SerializableConfigurationSuite { - public SerializableConfiguration compileTest() { - SerializableConfiguration scs = new SerializableConfiguration(null); - return scs; - } -} diff --git a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala index 52b309abd77f2..42d7f71404594 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala @@ -23,8 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Unstable} /** - * Helper wrapper to serialize a Hadoop configuration. Intended for use when implementing - * DataSourceV2 readers & writers which depend on the Hadoop configuration from the driver node. + * Hadoop configuration but serializable. Use `value` to access the Hadoop configuration. + * + * @param value Hadoop configuration */ @DeveloperApi @Unstable class SerializableConfiguration(@transient var value: Configuration) extends Serializable { diff --git a/core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java b/core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java new file mode 100644 index 0000000000000..0944d681599a1 --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util; + +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import static org.junit.Assert.assertEquals; + + +public class SerializableConfigurationSuite { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "SerializableConfigurationSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void testSerializableConfiguration() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + Configuration hadoopConfiguration = new Configuration(false); + hadoopConfiguration.set("test.property", "value"); + SerializableConfiguration scs = new SerializableConfiguration(hadoopConfiguration); + SerializableConfiguration actual = rdd.map(val -> scs).collect().get(0); + assertEquals(actual.value().get("test.property"), "value"); + } +} From 9e234a5434f874a9548123375df5bb7a1fee5bc5 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 20 Sep 2019 08:54:34 -0700 Subject: [PATCH 62/87] [MINOR][INFRA] Use java-version instead of version for GitHub Action ### What changes were proposed in this pull request? This PR use `java-version` instead of `version` for GitHub Action. More details: https://github.com/actions/setup-java/commit/204b974cf476e9709b6fab0c59007578676321c5 https://github.com/actions/setup-java/commit/ac25aeee3a8ad80e5e24d12610e451338577534f ### Why are the changes needed? The `version` property will not be supported after October 1, 2019. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25866 from wangyum/java-version. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 99dd1d04a60d1..350f3405bc4c7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -26,7 +26,7 @@ jobs: - name: Set up JDK ${{ matrix.java }} uses: actions/setup-java@v1 with: - version: ${{ matrix.java }} + java-version: ${{ matrix.java }} - name: Build with Maven run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" From 27d0c3f9131835cd7f992841dbcda890a7b67ec8 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 20 Sep 2019 08:57:47 -0700 Subject: [PATCH 63/87] [SPARK-29139][CORE][TESTS] Increase timeout to wait for executor(s) to be up in SparkContextSuite ### What changes were proposed in this pull request? This patch proposes to increase timeout to wait for executor(s) to be up in SparkContextSuite, as we observed these tests failed due to wait timeout. ### Why are the changes needed? There's some case that CI build is extremely slow which requires 3x or more time to pass the test. (https://issues.apache.org/jira/browse/SPARK-29139?focusedCommentId=16934034&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16934034) Allocating higher timeout wouldn't bring additional latency, as the code checks the condition with sleeping 10 ms per loop iteration. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A, as the case is not likely to be occurred frequently. Closes #25864 from HeartSaVioR/SPARK-29139. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index eca49e5efdbbb..a7fb7ea27ff26 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -761,7 +761,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) // Ensure all executors has started - TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) assert(sc.resources.size === 1) assert(sc.resources.get(GPU).get.addresses === Array("5", "6")) assert(sc.resources.get(GPU).get.name === "gpu") @@ -790,7 +790,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) // Ensure all executors has started - TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) // driver gpu resources file should take precedence over the script assert(sc.resources.size === 1) assert(sc.resources.get(GPU).get.addresses === Array("0", "1", "8")) From 42050c3f4f21adaa14808e474a0db69f62671935 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 20 Sep 2019 09:59:31 -0700 Subject: [PATCH 64/87] [SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator ### What changes were proposed in this pull request? This PR allows Python toLocalIterator to prefetch the next partition while the first partition is being collected. The PR also adds a demo micro bench mark in the examples directory, we may wish to keep this or not. ### Why are the changes needed? In https://issues.apache.org/jira/browse/SPARK-23961 / 5e79ae3b40b76e3473288830ab958fc4834dcb33 we changed PySpark to only pull one partition at a time. This is memory efficient, but if partitions take time to compute this can mean we're spending more time blocking. ### Does this PR introduce any user-facing change? A new param is added to toLocalIterator ### How was this patch tested? New unit test inside of `test_rdd.py` checks the time that the elements are evaluated at. Another test that the results remain the same are added to `test_dataframe.py`. I also ran a micro benchmark in the examples directory `prefetch.py` which shows an improvement of ~40% in this specific use case. > > 19/08/16 17:11:36 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable > Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties > Setting default log level to "WARN". > To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). > Running timers: > > [Stage 32:> (0 + 1) / 1] > Results: > > Prefetch time: > > 100.228110831 > > > Regular time: > > 188.341721614 > > > Closes #25515 from holdenk/SPARK-27659-allow-pyspark-tolocalitr-to-prefetch. Authored-by: Holden Karau Signed-off-by: Holden Karau --- .../apache/spark/api/python/PythonRDD.scala | 21 ++++++++++++++---- python/pyspark/rdd.py | 10 +++++++-- python/pyspark/sql/dataframe.py | 8 +++++-- python/pyspark/sql/tests/test_dataframe.py | 6 +++++ python/pyspark/tests/test_rdd.py | 22 +++++++++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 6 files changed, 61 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 4d76ff76e6752..7cbfb71beea37 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,6 +24,7 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.duration.Duration import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration @@ -179,15 +180,22 @@ private[spark] object PythonRDD extends Logging { * data collected from this job, the secret for authentication, and a socket auth * server object that can be used to join the JVM serving thread in Python. */ - def toLocalIteratorAndServe[T](rdd: RDD[T]): Array[Any] = { + def toLocalIteratorAndServe[T](rdd: RDD[T], prefetchPartitions: Boolean = false): Array[Any] = { val handleFunc = (sock: Socket) => { val out = new DataOutputStream(sock.getOutputStream) val in = new DataInputStream(sock.getInputStream) Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Collects a partition on each iteration val collectPartitionIter = rdd.partitions.indices.iterator.map { i => - rdd.sparkContext.runJob(rdd, (iter: Iterator[Any]) => iter.toArray, Seq(i)).head + var result: Array[Any] = null + rdd.sparkContext.submitJob( + rdd, + (iter: Iterator[Any]) => iter.toArray, + Seq(i), // The partition we are evaluating + (_, res: Array[Any]) => result = res, + result) } + val prefetchIter = collectPartitionIter.buffered // Write data until iteration is complete, client stops iteration, or error occurs var complete = false @@ -196,10 +204,15 @@ private[spark] object PythonRDD extends Logging { // Read request for data, value of zero will stop iteration or non-zero to continue if (in.readInt() == 0) { complete = true - } else if (collectPartitionIter.hasNext) { + } else if (prefetchIter.hasNext) { // Client requested more data, attempt to collect the next partition - val partitionArray = collectPartitionIter.next() + val partitionFuture = prefetchIter.next() + // Cause the next job to be submitted if prefetchPartitions is enabled. + if (prefetchPartitions) { + prefetchIter.headOption + } + val partitionArray = ThreadUtils.awaitResult(partitionFuture, Duration.Inf) // Send response there is a partition to read out.writeInt(1) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index be0244b7d13e2..1edffaa4ca168 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2437,17 +2437,23 @@ def countApproxDistinct(self, relativeSD=0.05): hashRDD = self.map(lambda x: portable_hash(x) & 0xFFFFFFFF) return hashRDD._to_java_object_rdd().countApproxDistinct(relativeSD) - def toLocalIterator(self): + def toLocalIterator(self, prefetchPartitions=False): """ Return an iterator that contains all of the elements in this RDD. The iterator will consume as much memory as the largest partition in this RDD. + With prefetch it may consume up to the memory of the 2 largest partitions. + + :param prefetchPartitions: If Spark should pre-fetch the next partition + before it is needed. >>> rdd = sc.parallelize(range(10)) >>> [x for x in rdd.toLocalIterator()] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ with SCCallSiteSync(self.context) as css: - sock_info = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe(self._jrdd.rdd()) + sock_info = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe( + self._jrdd.rdd(), + prefetchPartitions) return _local_iterator_from_socket(sock_info, self._jrdd_deserializer) def barrier(self): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 398471234d2b7..03b37fa7d0d9b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -520,16 +520,20 @@ def collect(self): @ignore_unicode_prefix @since(2.0) - def toLocalIterator(self): + def toLocalIterator(self, prefetchPartitions=False): """ Returns an iterator that contains all of the rows in this :class:`DataFrame`. The iterator will consume as much memory as the largest partition in this DataFrame. + With prefetch it may consume up to the memory of the 2 largest partitions. + + :param prefetchPartitions: If Spark should pre-fetch the next partition + before it is needed. >>> list(df.toLocalIterator()) [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ with SCCallSiteSync(self._sc) as css: - sock_info = self._jdf.toPythonIterator() + sock_info = self._jdf.toPythonIterator(prefetchPartitions) return _local_iterator_from_socket(sock_info, BatchedSerializer(PickleSerializer())) @ignore_unicode_prefix diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index bc4ee8814f7af..90a5415c15c27 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -690,6 +690,12 @@ def test_to_local_iterator(self): expected = df.collect() self.assertEqual(expected, list(it)) + def test_to_local_iterator_prefetch(self): + df = self.spark.range(8, numPartitions=4) + expected = df.collect() + it = df.toLocalIterator(prefetchPartitions=True) + self.assertEqual(expected, list(it)) + def test_to_local_iterator_not_fully_consumed(self): # SPARK-23961: toLocalIterator throws exception when not fully consumed # Create a DataFrame large enough so that write to socket will eventually block diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index bff080362085f..e7a7971dfc9a6 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -14,11 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. # +from datetime import datetime, timedelta import hashlib import os import random import sys import tempfile +import time from glob import glob from py4j.protocol import Py4JJavaError @@ -68,6 +70,26 @@ def test_to_localiterator(self): it2 = rdd2.toLocalIterator() self.assertEqual([1, 2, 3], sorted(it2)) + def test_to_localiterator_prefetch(self): + # Test that we fetch the next partition in parallel + # We do this by returning the current time and: + # reading the first elem, waiting, and reading the second elem + # If not in parallel then these would be at different times + # But since they are being computed in parallel we see the time + # is "close enough" to the same. + rdd = self.sc.parallelize(range(2), 2) + times1 = rdd.map(lambda x: datetime.now()) + times2 = rdd.map(lambda x: datetime.now()) + times_iter_prefetch = times1.toLocalIterator(prefetchPartitions=True) + times_iter = times2.toLocalIterator(prefetchPartitions=False) + times_prefetch_head = next(times_iter_prefetch) + times_head = next(times_iter) + time.sleep(2) + times_next = next(times_iter) + times_prefetch_next = next(times_iter_prefetch) + self.assertTrue(times_next - times_head >= timedelta(seconds=2)) + self.assertTrue(times_prefetch_next - times_prefetch_head < timedelta(seconds=1)) + def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" 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 a2f5f03d797a2..9a2d80030ee66 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 @@ -3356,9 +3356,9 @@ class Dataset[T] private[sql]( } } - private[sql] def toPythonIterator(): Array[Any] = { + private[sql] def toPythonIterator(prefetchPartitions: Boolean = false): Array[Any] = { withNewExecutionId { - PythonRDD.toLocalIteratorAndServe(javaToPython.rdd) + PythonRDD.toLocalIteratorAndServe(javaToPython.rdd, prefetchPartitions) } } From 4080c4beeb9cb27027145a37799ee8599ee51aab Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 20 Sep 2019 10:08:16 -0700 Subject: [PATCH 65/87] [SPARK-28937][SPARK-28936][KUBERNETES] Reduce test flakyness ### What changes were proposed in this pull request? Switch from using a Thread sleep for waiting for commands to finish to just waiting for the command to finish with a watcher & improve the error messages in the SecretsTestsSuite. ### Why are the changes needed? Currently some of the Spark Kubernetes tests have race conditions with command execution, and the frequent use of eventually makes debugging test failures difficult. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests pass after removal of thread.sleep Closes #25765 from holdenk/SPARK-28937SPARK-28936-improve-kubernetes-integration-tests. Authored-by: Holden Karau Signed-off-by: Holden Karau --- .../integrationtest/SecretsTestsSuite.scala | 51 +++++++++++++------ .../deploy/k8s/integrationtest/Utils.scala | 40 +++++++++++++-- 2 files changed, 72 insertions(+), 19 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index cd61ea1040f35..54a9dbf07816c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.k8s.integrationtest +import java.util.Locale + import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{Pod, SecretBuilder} @@ -57,11 +59,17 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => createTestSecret() sparkAppConf .set(s"spark.kubernetes.driver.secrets.$ENV_SECRET_NAME", SECRET_MOUNT_PATH) - .set(s"spark.kubernetes.driver.secretKeyRef.USERNAME", s"$ENV_SECRET_NAME:username") - .set(s"spark.kubernetes.driver.secretKeyRef.PASSWORD", s"$ENV_SECRET_NAME:password") + .set( + s"spark.kubernetes.driver.secretKeyRef.${ENV_SECRET_KEY_1_CAP}", + s"$ENV_SECRET_NAME:${ENV_SECRET_KEY_1}") + .set( + s"spark.kubernetes.driver.secretKeyRef.${ENV_SECRET_KEY_2_CAP}", + s"$ENV_SECRET_NAME:${ENV_SECRET_KEY_2}") .set(s"spark.kubernetes.executor.secrets.$ENV_SECRET_NAME", SECRET_MOUNT_PATH) - .set(s"spark.kubernetes.executor.secretKeyRef.USERNAME", s"$ENV_SECRET_NAME:username") - .set(s"spark.kubernetes.executor.secretKeyRef.PASSWORD", s"$ENV_SECRET_NAME:password") + .set(s"spark.kubernetes.executor.secretKeyRef.${ENV_SECRET_KEY_1_CAP}", + s"${ENV_SECRET_NAME}:$ENV_SECRET_KEY_1") + .set(s"spark.kubernetes.executor.secretKeyRef.${ENV_SECRET_KEY_2_CAP}", + s"${ENV_SECRET_NAME}:$ENV_SECRET_KEY_2") try { runSparkPiAndVerifyCompletion( driverPodChecker = (driverPod: Pod) => { @@ -81,19 +89,30 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => } private def checkSecrets(pod: Pod): Unit = { - Eventually.eventually(TIMEOUT, INTERVAL) { - implicit val podName: String = pod.getMetadata.getName - implicit val components: KubernetesTestComponents = kubernetesTestComponents + logDebug(s"Checking secrets for ${pod}") + // Wait for the pod to become ready & have secrets provisioned + implicit val podName: String = pod.getMetadata.getName + implicit val components: KubernetesTestComponents = kubernetesTestComponents + val env = Eventually.eventually(TIMEOUT, INTERVAL) { + logDebug(s"Checking env of ${pod.getMetadata().getName()} ....") val env = Utils.executeCommand("env") - assert(env.toString.contains(ENV_SECRET_VALUE_1)) - assert(env.toString.contains(ENV_SECRET_VALUE_2)) - val fileUsernameContents = Utils - .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") - val filePasswordContents = Utils - .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") - assert(fileUsernameContents.toString.trim.equals(ENV_SECRET_VALUE_1)) - assert(filePasswordContents.toString.trim.equals(ENV_SECRET_VALUE_2)) + assert(!env.isEmpty) + env } + env.toString should include (s"${ENV_SECRET_KEY_1_CAP}=$ENV_SECRET_VALUE_1") + env.toString should include (s"${ENV_SECRET_KEY_2_CAP}=$ENV_SECRET_VALUE_2") + + // Make sure our secret files are mounted correctly + val files = Utils.executeCommand("ls", s"$SECRET_MOUNT_PATH") + files should include (ENV_SECRET_KEY_1) + files should include (ENV_SECRET_KEY_2) + // Validate the contents + val fileUsernameContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") + fileUsernameContents.toString.trim should equal(ENV_SECRET_VALUE_1) + val filePasswordContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") + filePasswordContents.toString.trim should equal(ENV_SECRET_VALUE_2) } } @@ -102,6 +121,8 @@ private[spark] object SecretsTestsSuite { val SECRET_MOUNT_PATH = "/etc/secret" val ENV_SECRET_KEY_1 = "username" val ENV_SECRET_KEY_2 = "password" + val ENV_SECRET_KEY_1_CAP = ENV_SECRET_KEY_1.toUpperCase(Locale.ROOT) + val ENV_SECRET_KEY_2_CAP = ENV_SECRET_KEY_2.toUpperCase(Locale.ROOT) val ENV_SECRET_VALUE_1 = "secretusername" val ENV_SECRET_VALUE_2 = "secretpassword" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala index a687a1bca1800..9f85805b9d315 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala @@ -18,9 +18,12 @@ package org.apache.spark.deploy.k8s.integrationtest import java.io.{Closeable, File, PrintWriter} import java.nio.file.{Files, Path} +import java.util.concurrent.CountDownLatch import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.client.dsl.ExecListener +import okhttp3.Response import org.apache.commons.io.output.ByteArrayOutputStream import org.apache.spark.{SPARK_VERSION, SparkException} @@ -45,20 +48,49 @@ object Utils extends Logging { implicit podName: String, kubernetesTestComponents: KubernetesTestComponents): String = { val out = new ByteArrayOutputStream() - val watch = kubernetesTestComponents + val pod = kubernetesTestComponents .kubernetesClient .pods() .withName(podName) + // Avoid timing issues by looking for open/close + class ReadyListener extends ExecListener { + val openLatch: CountDownLatch = new CountDownLatch(1) + val closeLatch: CountDownLatch = new CountDownLatch(1) + + override def onOpen(response: Response) { + openLatch.countDown() + } + + override def onClose(a: Int, b: String) { + closeLatch.countDown() + } + + override def onFailure(e: Throwable, r: Response) { + } + + def waitForInputStreamToConnect(): Unit = { + openLatch.await() + } + + def waitForClose(): Unit = { + closeLatch.await() + } + } + val listener = new ReadyListener() + val watch = pod .readingInput(System.in) .writingOutput(out) .writingError(System.err) .withTTY() + .usingListener(listener) .exec(cmd.toArray: _*) - // wait to get some result back - Thread.sleep(1000) + // under load sometimes the stdout isn't connected by the time we try to read from it. + listener.waitForInputStreamToConnect() + listener.waitForClose() watch.close() out.flush() - out.toString() + val result = out.toString() + result } def createTempFile(contents: String, hostPath: String): String = { From 8c8016a152a7f1dce96d33994a3b541c0bb06f00 Mon Sep 17 00:00:00 2001 From: Xianjin YE Date: Sat, 21 Sep 2019 08:09:19 +0900 Subject: [PATCH 66/87] [SPARK-21045][PYTHON] Allow non-ascii string as an exception message from python execution in Python 2 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR allows non-ascii string as an exception message in Python 2 by explicitly en/decoding in case of `str` in Python 2. ### Why are the changes needed? Previously PySpark will hang when the `UnicodeDecodeError` occurs and the real exception cannot be passed to the JVM side. See the reproducer as below: ```python def f(): raise Exception("中") spark = SparkSession.builder.master('local').getOrCreate() spark.sparkContext.parallelize([1]).map(lambda x: f()).count() ``` ### Does this PR introduce any user-facing change? User may not observe hanging for the similar cases. ### How was this patch tested? Added a new test and manually checking. This pr is based on #18324, credits should also go to dataknocker. To make lint-python happy for python3, it also includes a followup fix for #25814 Closes #25847 from advancedxy/python_exception_19926_and_21045. Authored-by: Xianjin YE Signed-off-by: HyukjinKwon --- python/pyspark/sql/utils.py | 3 +++ python/pyspark/tests/test_worker.py | 15 +++++++++++++++ python/pyspark/worker.py | 9 ++++++++- 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 996b7dd59ce9f..83afafdd8b138 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -18,6 +18,9 @@ import py4j import sys +if sys.version_info.major >= 3: + unicode = str + class CapturedException(Exception): def __init__(self, desc, stackTrace, cause=None): diff --git a/python/pyspark/tests/test_worker.py b/python/pyspark/tests/test_worker.py index 18fde17f4a064..ccbe21f3a6f38 100644 --- a/python/pyspark/tests/test_worker.py +++ b/python/pyspark/tests/test_worker.py @@ -1,3 +1,4 @@ +# -*- encoding: utf-8 -*- # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -150,6 +151,20 @@ def test_with_different_versions_of_python(self): finally: self.sc.pythonVer = version + def test_python_exception_non_hanging(self): + # SPARK-21045: exceptions with no ascii encoding shall not hanging PySpark. + try: + def f(): + raise Exception("exception with 中 and \xd6\xd0") + + self.sc.parallelize([1]).map(lambda x: f()).count() + except Py4JJavaError as e: + if sys.version_info.major < 3: + # we have to use unicode here to avoid UnicodeDecodeError + self.assertRegexpMatches(unicode(e).encode("utf-8"), "exception with 中") + else: + self.assertRegexpMatches(str(e), "exception with 中") + class WorkerReuseTest(PySparkTestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 086202de2c68b..698193d6bdd87 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -598,8 +598,15 @@ def process(): process() except Exception: try: + exc_info = traceback.format_exc() + if isinstance(exc_info, bytes): + # exc_info may contains other encoding bytes, replace the invalid bytes and convert + # it back to utf-8 again + exc_info = exc_info.decode("utf-8", "replace").encode("utf-8") + else: + exc_info = exc_info.encode("utf-8") write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc().encode("utf-8"), outfile) + write_with_length(exc_info, outfile) except IOError: # JVM close the socket pass From abc88deeed164bf48eff1d1504e141c1cf5afade Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 21 Sep 2019 08:26:30 +0900 Subject: [PATCH 67/87] [SPARK-29063][SQL] Modify fillValue approach to support joined dataframe ### What changes were proposed in this pull request? Modify the approach in `DataFrameNaFunctions.fillValue`, the new one uses `df.withColumns` which only address the columns need to be filled. After this change, there are no more ambiguous fileds detected for joined dataframe. ### Why are the changes needed? Before this change, when you have a joined table that has the same field name from both original table, fillna will fail even if you specify a subset that does not include the 'ambiguous' fields. ``` scala> val df1 = Seq(("f1-1", "f2", null), ("f1-2", null, null), ("f1-3", "f2", "f3-1"), ("f1-4", "f2", "f3-1")).toDF("f1", "f2", "f3") scala> val df2 = Seq(("f1-1", null, null), ("f1-2", "f2", null), ("f1-3", "f2", "f4-1")).toDF("f1", "f2", "f4") scala> val df_join = df1.alias("df1").join(df2.alias("df2"), Seq("f1"), joinType="left_outer") scala> df_join.na.fill("", cols=Seq("f4")) org.apache.spark.sql.AnalysisException: Reference 'f2' is ambiguous, could be: df1.f2, df2.f2.; ``` ### Does this PR introduce any user-facing change? Yes, fillna operation will pass and give the right answer for a joined table. ### How was this patch tested? Local test and newly added UT. Closes #25768 from xuanyuanking/SPARK-29063. Lead-authored-by: Yuanjian Li Co-authored-by: Xiao Li Signed-off-by: HyukjinKwon --- .../spark/sql/DataFrameNaFunctions.scala | 10 ++---- .../spark/sql/DataFrameNaFunctionsSuite.scala | 35 +++++++++++++++++++ 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 53e9f810d7c85..6dd21f114c902 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -488,7 +488,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { } val columnEquals = df.sparkSession.sessionState.analyzer.resolver - val projections = df.schema.fields.map { f => + val filledColumns = df.schema.fields.filter { f => val typeMatches = (targetType, f.dataType) match { case (NumericType, dt) => dt.isInstanceOf[NumericType] case (StringType, dt) => dt == StringType @@ -497,12 +497,8 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { throw new IllegalArgumentException(s"$targetType is not matched at fillValue") } // Only fill if the column is part of the cols list. - if (typeMatches && cols.exists(col => columnEquals(f.name, col))) { - fillCol[T](f, value) - } else { - df.col(f.name) - } + typeMatches && cols.exists(col => columnEquals(f.name, col)) } - df.select(projections : _*) + df.withColumns(filledColumns.map(_.name), filledColumns.map(fillCol[T](_, value))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index aeee4577d3483..75642a0bd9325 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -231,6 +231,41 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { } } + def createDFsWithSameFieldsName(): (DataFrame, DataFrame) = { + val df1 = Seq( + ("f1-1", "f2", null), + ("f1-2", null, null), + ("f1-3", "f2", "f3-1"), + ("f1-4", "f2", "f3-1") + ).toDF("f1", "f2", "f3") + val df2 = Seq( + ("f1-1", null, null), + ("f1-2", "f2", null), + ("f1-3", "f2", "f4-1") + ).toDF("f1", "f2", "f4") + (df1, df2) + } + + test("fill unambiguous field for join operation") { + val (df1, df2) = createDFsWithSameFieldsName() + val joined_df = df1.join(df2, Seq("f1"), joinType = "left_outer") + checkAnswer(joined_df.na.fill("", cols = Seq("f4")), + Row("f1-1", "f2", null, null, "") :: + Row("f1-2", null, null, "f2", "") :: + Row("f1-3", "f2", "f3-1", "f2", "f4-1") :: + Row("f1-4", "f2", "f3-1", null, "") :: Nil) + } + + test("fill ambiguous field for join operation") { + val (df1, df2) = createDFsWithSameFieldsName() + val joined_df = df1.join(df2, Seq("f1"), joinType = "left_outer") + + val message = intercept[AnalysisException] { + joined_df.na.fill("", cols = Seq("f2")) + }.getMessage + assert(message.contains("Reference 'f2' is ambiguous")) + } + test("replace") { val input = createDF() From 4a89fa1cd1d3f25abc02be83a90653f465a452aa Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 21 Sep 2019 08:40:49 +0900 Subject: [PATCH 68/87] [SPARK-29196][DOCS] Add JDK11 support to the document ### What changes were proposed in this pull request? This PRs add Java 11 version to the document. ### Why are the changes needed? Apache Spark 3.0.0 starts to support JDK11 officially. ### Does this PR introduce any user-facing change? Yes. ![jdk11](https://user-images.githubusercontent.com/9700541/65364063-39204580-dbc4-11e9-982b-fc1552be2ec5.png) ### How was this patch tested? Manually. Doc generation. Closes #25875 from dongjoon-hyun/SPARK-29196. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- docs/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/index.md b/docs/index.md index 6cfbbaed20028..edb1c421fb794 100644 --- a/docs/index.md +++ b/docs/index.md @@ -46,7 +46,7 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. -Spark runs on Java 8, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. +Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. Python 2 support is deprecated as of Spark 3.0.0. R prior to version 3.4 support is deprecated as of Spark 3.0.0. For the Scala API, Spark {{site.SPARK_VERSION}} From c764dd6dd725647d3dca2493875429513f706e86 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 20 Sep 2019 19:22:46 -0500 Subject: [PATCH 69/87] [SPARK-29144][ML] Binarizer handle sparse vectors incorrectly with negative threshold ### What changes were proposed in this pull request? if threshold<0, convert implict 0 to 1, althought this will break sparsity ### Why are the changes needed? if `threshold<0`, current impl deal with sparse vector incorrectly. See JIRA [SPARK-29144](https://issues.apache.org/jira/browse/SPARK-29144) and [Scikit-Learn's Binarizer](https://scikit-learn.org/stable/modules/generated/sklearn.preprocessing.Binarizer.html) ('Threshold may not be less than 0 for operations on sparse matrices.') for details. ### Does this PR introduce any user-facing change? no ### How was this patch tested? added testsuite Closes #25829 from zhengruifeng/binarizer_throw_exception_sparse_vector. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../apache/spark/ml/feature/Binarizer.scala | 48 +++++++++++-------- .../spark/ml/feature/BinarizerSuite.scala | 14 ++++++ 2 files changed, 43 insertions(+), 19 deletions(-) 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 2b0862c60fdf7..c4daf64dfc5f0 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 @@ -75,30 +75,40 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) val schema = dataset.schema val inputType = schema($(inputCol)).dataType val td = $(threshold) + val metadata = outputSchema($(outputCol)).metadata - val binarizerDouble = udf { in: Double => if (in > td) 1.0 else 0.0 } - val binarizerVector = udf { (data: Vector) => - val indices = ArrayBuilder.make[Int] - val values = ArrayBuilder.make[Double] - - data.foreachActive { (index, value) => - if (value > td) { - indices += index - values += 1.0 + 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 + } + } + Vectors.sparse(vector.size, indices.result(), values.result()).compressed } - } - Vectors.sparse(data.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 + } + } + Vectors.dense(values).compressed + } } - val metadata = outputSchema($(outputCol)).metadata - - inputType match { - case DoubleType => - dataset.select(col("*"), binarizerDouble(col($(inputCol))).as($(outputCol), metadata)) - case _: VectorUDT => - dataset.select(col("*"), binarizerVector(col($(inputCol))).as($(outputCol), metadata)) - } + dataset.withColumn($(outputCol), binarizerUDF(col($(inputCol))), metadata) } @Since("1.4.0") 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 05d4a6ee2dabf..91bec50fb904f 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 @@ -101,6 +101,20 @@ class BinarizerSuite extends MLTest with DefaultReadWriteTest { } } + test("Binarizer should support sparse vector with negative threshold") { + val data = Seq( + (Vectors.sparse(3, Array(1), Array(0.5)), Vectors.dense(Array(1.0, 1.0, 1.0))), + (Vectors.dense(Array(0.0, 0.5, 0.0)), Vectors.dense(Array(1.0, 1.0, 1.0)))) + val df = data.toDF("feature", "expected") + val binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + .setThreshold(-0.5) + binarizer.transform(df).select("binarized_feature", "expected").collect().foreach { + case Row(x: Vector, y: Vector) => + assert(x == y, "The feature value is not correct after binarization.") + } + } test("read/write") { val t = new Binarizer() From ff3a737c759b21d166053a0fbb6ad539accb8e49 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 20 Sep 2019 19:41:25 -0700 Subject: [PATCH 70/87] [SPARK-29192][TESTS] Extend BenchmarkBase to write JDK9+ results separately ### What changes were proposed in this pull request? This PR aims to extend the existing benchmarks to save JDK9+ result separately. All `core` module benchmark test results are added. I'll run the other test suites in another PR. After regenerating all results, we will check JDK11 performance regressions. ### Why are the changes needed? From Apache Spark 3.0, we support both JDK8 and JDK11. We need to have a way to find the performance regression. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually run the benchmark. Closes #25873 from dongjoon-hyun/SPARK-JDK11-PERF. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../CoalescedRDDBenchmark-jdk11-results.txt | 40 +++++++++++++++++ .../KryoBenchmark-jdk11-results.txt | 28 ++++++++++++ .../KryoSerializerBenchmark-jdk11-results.txt | 12 +++++ ...PropertiesCloneBenchmark-jdk11-results.txt | 40 +++++++++++++++++ .../XORShiftRandomBenchmark-jdk11-results.txt | 44 +++++++++++++++++++ .../spark/benchmark/BenchmarkBase.scala | 5 ++- 6 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/KryoBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt diff --git a/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt b/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..f43e083060e34 --- /dev/null +++ b/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Coalesced RDD , large scale +================================================================================================ + +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 + + diff --git a/core/benchmarks/KryoBenchmark-jdk11-results.txt b/core/benchmarks/KryoBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0afa06519d2f4 --- /dev/null +++ b/core/benchmarks/KryoBenchmark-jdk11-results.txt @@ -0,0 +1,28 @@ +================================================================================================ +Benchmark Kryo Unsafe vs safe 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 +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 + + diff --git a/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt b/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0e6d32649840f --- /dev/null +++ b/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt @@ -0,0 +1,12 @@ +================================================================================================ +Benchmark KryoPool vs old"pool of 1" implementation +================================================================================================ + +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 + + diff --git a/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt b/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..1cd08a7748109 --- /dev/null +++ b/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Properties Cloning +================================================================================================ + +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 + +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 + +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 + +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 + +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 + + diff --git a/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt b/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..3b6091cc740ac --- /dev/null +++ b/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt @@ -0,0 +1,44 @@ +================================================================================================ +Pseudo random +================================================================================================ + +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 + +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 + +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 + +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 + + +================================================================================================ +hash seed +================================================================================================ + +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 + + diff --git a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala index a6666db4e95c3..55e34b32fe0d4 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala @@ -21,6 +21,7 @@ import java.io.{File, FileOutputStream, OutputStream} /** * A base class for generate benchmark results to a file. + * For JDK9+, JDK major version number is added to the file names to distingush the results. */ abstract class BenchmarkBase { var output: Option[OutputStream] = None @@ -43,7 +44,9 @@ abstract class BenchmarkBase { def main(args: Array[String]): Unit = { val regenerateBenchmarkFiles: Boolean = System.getenv("SPARK_GENERATE_BENCHMARK_FILES") == "1" if (regenerateBenchmarkFiles) { - val resultFileName = s"${this.getClass.getSimpleName.replace("$", "")}-results.txt" + val version = System.getProperty("java.version").split("\\D+")(0).toInt + val jdkString = if (version > 8) s"-jdk$version" else "" + val resultFileName = s"${this.getClass.getSimpleName.replace("$", "")}$jdkString-results.txt" val file = new File(s"benchmarks/$resultFileName") if (!file.exists()) { file.createNewFile() From 252b6cf3c926724ba5c4ddc3659df2ad20bb4734 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Sep 2019 20:28:56 -0700 Subject: [PATCH 71/87] [SPARK-29187][SQL] Return null from `date_part()` for the null `field` ### What changes were proposed in this pull request? In the PR, I propose to change behavior of the `date_part()` function in handling `null` field, and make it the same as PostgreSQL has. If `field` parameter is `null`, the function should return `null` of the `double` type as PostgreSQL does: ```sql # select date_part(null, date '2019-09-20'); date_part ----------- (1 row) # select pg_typeof(date_part(null, date '2019-09-20')); pg_typeof ------------------ double precision (1 row) ``` ### Why are the changes needed? The `date_part()` function was added to maintain feature parity with PostgreSQL but current behavior of the function is different in handling null as `field`. ### Does this PR introduce any user-facing change? Yes. Before: ```sql spark-sql> select date_part(null, date'2019-09-20'); Error in query: null; line 1 pos 7 ``` After: ```sql spark-sql> select date_part(null, date'2019-09-20'); NULL ``` ### How was this patch tested? Add new tests to `DateFunctionsSuite for 2 cases: - `field` = `null`, `source` = a date literal - `field` = `null`, `source` = a date column Closes #25865 from MaxGekk/date_part-null. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/datetimeExpressions.scala | 13 +++++++++---- .../test/resources/sql-tests/inputs/date_part.sql | 2 ++ .../resources/sql-tests/results/date_part.sql.out | 10 +++++++++- .../org/apache/spark/sql/DateFunctionsSuite.scala | 10 ++++++++++ 4 files changed, 30 insertions(+), 5 deletions(-) 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 89a6d23b1d73d..592b9de83d9a0 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 @@ -2053,10 +2053,15 @@ case class DatePart(field: Expression, source: Expression, child: Expression) if (!field.foldable) { throw new AnalysisException("The field parameter needs to be a foldable string value.") } - val fieldStr = field.eval().asInstanceOf[UTF8String].toString - DatePart.parseExtractField(fieldStr, source, { - throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") - }) + val fieldEval = field.eval() + if (fieldEval == null) { + 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.") + }) + } }) } 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 cb3d966281009..fd0fb50f71460 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 @@ -66,3 +66,5 @@ select date_part('secs', c) from t; select date_part('not_supported', c) from t; select date_part(c, c) from t; + +select date_part(null, c) from t; 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 c59dfdbd3da34..776786850e9da 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: 51 +-- Number of queries: 52 -- !query 0 @@ -410,3 +410,11 @@ struct<> -- !query 50 output org.apache.spark.sql.AnalysisException The field parameter needs to be a foldable string value.;; line 1 pos 7 + + +-- !query 51 +select date_part(null, c) from t +-- !query 51 schema +struct +-- !query 51 output +NULL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 2fef05f97e57c..99189a96b2995 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} import org.apache.spark.unsafe.types.CalendarInterval class DateFunctionsSuite extends QueryTest with SharedSparkSession { @@ -796,4 +797,13 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { Seq(Row(Instant.parse(timestamp)))) } } + + test("handling null field by date_part") { + val input = Seq(Date.valueOf("2019-09-20")).toDF("d") + Seq("date_part(null, d)", "date_part(null, date'2019-09-20')").foreach { expr => + val df = input.selectExpr(expr) + assert(df.schema.headOption.get.dataType == DoubleType) + checkAnswer(df, Row(null)) + } + } } From a9ae262cf279bc607cb842204c717257c259d82b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 20 Sep 2019 20:31:26 -0700 Subject: [PATCH 72/87] [SPARK-28772][BUILD][MLLIB] Update breeze to 1.0 ### What changes were proposed in this pull request? Update breeze dependency to 1.0. ### Why are the changes needed? Breeze 1.0 supports Scala 2.13 and has a few bug fixes. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #25874 from srowen/SPARK-28772. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 15 +- dev/deps/spark-deps-hadoop-2.7 | 20 +- dev/deps/spark-deps-hadoop-3.2 | 20 +- licenses-binary/LICENSE-JLargeArrays.txt | 23 ++ licenses-binary/LICENSE-JTransforms.txt | 23 ++ licenses-binary/LICENSE-jtransforms.html | 388 ------------------ .../org/apache/spark/ml/feature/DCT.scala | 2 +- .../apache/spark/ml/feature/JavaDCTSuite.java | 2 +- .../apache/spark/ml/feature/DCTSuite.scala | 2 +- pom.xml | 8 +- 10 files changed, 84 insertions(+), 419 deletions(-) create mode 100644 licenses-binary/LICENSE-JLargeArrays.txt create mode 100644 licenses-binary/LICENSE-JTransforms.txt delete mode 100644 licenses-binary/LICENSE-jtransforms.html diff --git a/LICENSE-binary b/LICENSE-binary index ba20eea118687..16da39145d050 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -396,6 +396,8 @@ org.codehaus.janino:commons-compiler org.codehaus.janino:janino jline:jline org.jodd:jodd-core +com.github.wendykierp:JTransforms +pl.edu.icm:JLargeArrays BSD 3-Clause @@ -433,8 +435,12 @@ is distributed under the 3-Clause BSD license. MIT License ----------- -org.spire-math:spire-macros_2.12 -org.spire-math:spire_2.12 +org.typelevel:spire_2.12 +org.typelevel:spire-macros_2.12 +org.typelevel:spire-platform_2.12 +org.typelevel:spire-util_2.12 +org.typelevel:algebra_2.12:jar +org.typelevel:cats-kernel_2.12 org.typelevel:machinist_2.12 net.razorvine:pyrolite org.slf4j:jcl-over-slf4j @@ -496,11 +502,6 @@ 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 -Mozilla Public License (MPL) 1.1 --------------------------------- - -com.github.rwl:jtransforms https://sourceforge.net/projects/jtransforms/ - Python Software Foundation License ---------------------------------- diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index cae2614128b1a..2da4c9e44b29e 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -1,8 +1,11 @@ +JLargeArrays-1.5.jar +JTransforms-3.1.jar JavaEWAH-0.3.2.jar RoaringBitmap-0.7.45.jar ST4-4.0.4.jar activation-1.1.1.jar aircompressor-0.10.jar +algebra_2.12-2.0.0-M2.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar @@ -22,8 +25,9 @@ avro-1.8.2.jar avro-ipc-1.8.2.jar avro-mapred-1.8.2-hadoop2.jar bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-0.13.2.jar -breeze_2.12-0.13.2.jar +breeze-macros_2.12-1.0.jar +breeze_2.12-1.0.jar +cats-kernel_2.12-2.0.0-M4.jar chill-java-0.9.3.jar chill_2.12-0.9.3.jar commons-beanutils-1.9.3.jar @@ -130,7 +134,6 @@ json4s-scalap_2.12-3.6.6.jar jsp-api-2.1.jar jsr305-3.0.0.jar jta-1.1.jar -jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-4.0.2.jar kubernetes-client-4.4.2.jar @@ -142,7 +145,7 @@ libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar lz4-java-1.6.0.jar -machinist_2.12-0.6.1.jar +machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar metrics-core-3.1.5.jar @@ -172,20 +175,23 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar +scala-collection-compat_2.12-2.1.1.jar scala-compiler-2.12.10.jar scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.2.jar +shapeless_2.12-2.3.3.jar shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.23.jar snappy-0.2.jar snappy-java-1.1.7.3.jar -spire-macros_2.12-0.13.0.jar -spire_2.12-0.13.0.jar +spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12-0.17.0-M1.jar +spire_2.12-0.17.0-M1.jar stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.9.6.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 21df5a70e7f8d..2226baeadfba1 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -1,9 +1,12 @@ +JLargeArrays-1.5.jar +JTransforms-3.1.jar JavaEWAH-0.3.2.jar RoaringBitmap-0.7.45.jar ST4-4.0.4.jar accessors-smart-1.2.jar activation-1.1.1.jar aircompressor-0.10.jar +algebra_2.12-2.0.0-M2.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar @@ -20,8 +23,9 @@ avro-1.8.2.jar avro-ipc-1.8.2.jar avro-mapred-1.8.2-hadoop2.jar bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-0.13.2.jar -breeze_2.12-0.13.2.jar +breeze-macros_2.12-1.0.jar +breeze_2.12-1.0.jar +cats-kernel_2.12-2.0.0-M4.jar chill-java-0.9.3.jar chill_2.12-0.9.3.jar commons-beanutils-1.9.3.jar @@ -132,7 +136,6 @@ json4s-scalap_2.12-3.6.6.jar jsp-api-2.1.jar jsr305-3.0.0.jar jta-1.1.jar -jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kerb-admin-1.0.1.jar kerb-client-1.0.1.jar @@ -158,7 +161,7 @@ libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar lz4-java-1.6.0.jar -machinist_2.12-0.6.1.jar +machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar metrics-core-3.1.5.jar @@ -191,20 +194,23 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar re2j-1.1.jar +scala-collection-compat_2.12-2.1.1.jar scala-compiler-2.12.10.jar scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.2.jar +shapeless_2.12-2.3.3.jar shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.23.jar snappy-0.2.jar snappy-java-1.1.7.3.jar -spire-macros_2.12-0.13.0.jar -spire_2.12-0.13.0.jar +spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12-0.17.0-M1.jar +spire_2.12-0.17.0-M1.jar stax-api-1.0.1.jar stax2-api-3.1.4.jar stream-2.9.6.jar diff --git a/licenses-binary/LICENSE-JLargeArrays.txt b/licenses-binary/LICENSE-JLargeArrays.txt new file mode 100644 index 0000000000000..304e724556984 --- /dev/null +++ b/licenses-binary/LICENSE-JLargeArrays.txt @@ -0,0 +1,23 @@ +JLargeArrays +Copyright (C) 2013 onward University of Warsaw, ICM +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. 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 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/licenses-binary/LICENSE-JTransforms.txt b/licenses-binary/LICENSE-JTransforms.txt new file mode 100644 index 0000000000000..2f0589f76da7d --- /dev/null +++ b/licenses-binary/LICENSE-JTransforms.txt @@ -0,0 +1,23 @@ +JTransforms +Copyright (c) 2007 onward, Piotr Wendykier +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. 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 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/licenses-binary/LICENSE-jtransforms.html b/licenses-binary/LICENSE-jtransforms.html deleted file mode 100644 index 351c17412357b..0000000000000 --- a/licenses-binary/LICENSE-jtransforms.html +++ /dev/null @@ -1,388 +0,0 @@ - - -Mozilla Public License version 1.1 - - - - -

    Mozilla Public License Version 1.1

    -

    1. Definitions.

    -
    -
    1.0.1. "Commercial Use" -
    means distribution or otherwise making the Covered Code available to a third party. -
    1.1. "Contributor" -
    means each entity that creates or contributes to the creation of Modifications. -
    1.2. "Contributor Version" -
    means the combination of the Original Code, prior Modifications used by a Contributor, - and the Modifications made by that particular Contributor. -
    1.3. "Covered Code" -
    means the Original Code or Modifications or the combination of the Original Code and - Modifications, in each case including portions thereof. -
    1.4. "Electronic Distribution Mechanism" -
    means a mechanism generally accepted in the software development community for the - electronic transfer of data. -
    1.5. "Executable" -
    means Covered Code in any form other than Source Code. -
    1.6. "Initial Developer" -
    means the individual or entity identified as the Initial Developer in the Source Code - notice required by Exhibit A. -
    1.7. "Larger Work" -
    means a work which combines Covered Code or portions thereof with code not governed - by the terms of this License. -
    1.8. "License" -
    means this document. -
    1.8.1. "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 any addition to or deletion from the substance or structure of either the - Original Code or any previous Modifications. When Covered Code is released as a - series of files, a Modification is: -

      -
    1. Any addition to or deletion from the contents of a file - containing Original Code or previous Modifications. -
    2. Any new file that contains any part of the Original Code or - previous Modifications. -
    -
    1.10. "Original Code" -
    means Source Code of computer software code which is described in the Source Code - notice required by Exhibit A as Original Code, and which, - at the time of its release under this License is not already Covered Code governed - by this License. -
    1.10.1. "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.11. "Source Code" -
    means the preferred form of the Covered Code for making modifications to it, - including all modules it contains, plus any associated interface definition files, - scripts used to control compilation and installation of an Executable, or source - code differential comparisons against either the Original Code or another well known, - available Covered Code of the Contributor's choice. The Source Code can be in a - compressed or archival form, provided the appropriate decompression or de-archiving - software is widely available for no charge. -
    1.12. "You" (or "Your") -
    means an individual or a legal entity exercising rights under, and complying with - all of the terms of, this License or a future version of this License issued under - Section 6.1. 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. Source Code License.

    -

    2.1. The Initial Developer Grant.

    -

    The Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive - license, subject to third party intellectual property claims: -

      -
    1. under intellectual property rights (other than patent or - trademark) Licensable by Initial Developer to use, reproduce, modify, display, perform, - sublicense and distribute the Original Code (or portions thereof) with or without - Modifications, and/or as part of a Larger Work; and -
    2. under Patents Claims infringed by the making, using or selling - of Original Code, to make, have made, use, practice, sell, and offer for sale, and/or - otherwise dispose of the Original Code (or portions thereof). -
    3. the licenses granted in this Section 2.1 - (a) and (b) are effective on - the date Initial Developer first distributes Original Code under the terms of this - License. -
    4. Notwithstanding Section 2.1 (b) - above, no patent license is granted: 1) for code that You delete from the Original Code; - 2) separate from the Original Code; or 3) for infringements caused by: i) the - modification of the Original Code or ii) the combination of the Original Code with other - software or devices. -
    -

    2.2. Contributor Grant.

    -

    Subject to third party intellectual property claims, each Contributor hereby grants You - a world-wide, royalty-free, non-exclusive license -

      -
    1. 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 Code and/or as part of a Larger - Work; and -
    2. 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). -
    3. the licenses granted in Sections 2.2 - (a) and 2.2 (b) are effective - on the date Contributor first makes Commercial Use of the Covered Code. -
    4. Notwithstanding Section 2.2 (b) - above, no patent license is granted: 1) for any code that Contributor has deleted from - the Contributor Version; 2) separate from the Contributor Version; 3) 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 4) under Patent Claims infringed by Covered Code - in the absence of Modifications made by that Contributor. -
    -

    3. Distribution Obligations.

    -

    3.1. Application of License.

    -

    The Modifications which You create or to which You contribute are governed by the terms - of this License, including without limitation Section 2.2. The - Source Code version of Covered Code may be distributed only under the terms of this License - or a future version of this License released under Section 6.1, - and You must include a copy of this License with every copy of the Source Code You - distribute. You may not offer or impose any terms on any Source Code version that alters or - restricts the applicable version of this License or the recipients' rights hereunder. - However, You may include an additional document offering the additional rights described in - Section 3.5. -

    3.2. Availability of Source Code.

    -

    Any Modification which You create or to which You contribute must be made available in - Source Code form under the terms of this License either on the same media as an Executable - version or via an accepted Electronic Distribution Mechanism to anyone to whom you made an - Executable version available; and if made available via Electronic Distribution Mechanism, - must remain available for at least twelve (12) months after the date it initially became - available, or at least six (6) months after a subsequent version of that particular - Modification has been made available to such recipients. You are responsible for ensuring - that the Source Code version remains available even if the Electronic Distribution - Mechanism is maintained by a third party. -

    3.3. Description of Modifications.

    -

    You must cause all Covered Code to which You contribute to contain a file documenting the - changes You made to create that Covered Code and the date of any change. You must include a - prominent statement that the Modification is derived, directly or indirectly, from Original - Code provided by the Initial Developer and including the name of the Initial Developer in - (a) the Source Code, and (b) in any notice in an Executable version or related documentation - in which You describe the origin or ownership of the Covered Code. -

    3.4. Intellectual Property Matters

    -

    (a) Third Party Claims

    -

    If Contributor has knowledge that a license under a third party's intellectual property - rights is required to exercise the rights granted by such Contributor under Sections - 2.1 or 2.2, Contributor must include a - text file with the Source Code distribution titled "LEGAL" which describes the claim and the - party making the claim in sufficient detail that a recipient will know whom to contact. If - Contributor obtains such knowledge after the Modification is made available as described in - Section 3.2, Contributor shall promptly modify the LEGAL file in - all copies Contributor makes available thereafter and shall take other steps (such as - notifying appropriate mailing lists or newsgroups) reasonably calculated to inform those who - received the Covered Code that new knowledge has been obtained. -

    (b) Contributor APIs

    -

    If Contributor's Modifications include an application programming interface and Contributor - has knowledge of patent licenses which are reasonably necessary to implement that - API, Contributor must also include this information in the - legal file. -

    (c) Representations.

    -

    Contributor represents that, except as disclosed pursuant to Section 3.4 - (a) above, Contributor believes that Contributor's Modifications - are Contributor's original creation(s) and/or Contributor has sufficient rights to grant the - rights conveyed by this License. -

    3.5. Required Notices.

    -

    You must duplicate the notice in Exhibit A in each file of the - Source Code. If it is not possible to put such notice in a particular Source Code file due to - its structure, then You must include such notice in a location (such as a relevant directory) - where a user would be likely to look for such a notice. If You created one or more - Modification(s) You may add your name as a Contributor to the notice described in - Exhibit A. You must also duplicate this License in any documentation - for the Source Code where You describe recipients' rights or ownership rights relating to - Covered Code. You may choose to offer, and to charge a fee for, warranty, support, indemnity - or liability obligations to one or more recipients of Covered Code. 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 than 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.6. Distribution of Executable Versions.

    -

    You may distribute Covered Code in Executable form only if the requirements of Sections - 3.1, 3.2, - 3.3, 3.4 and - 3.5 have been met for that Covered Code, and if You include a - notice stating that the Source Code version of the Covered Code is available under the terms - of this License, including a description of how and where You have fulfilled the obligations - of Section 3.2. The notice must be conspicuously included in any - notice in an Executable version, related documentation or collateral in which You describe - recipients' rights relating to the Covered Code. You may distribute the Executable version of - Covered Code or ownership rights under 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 version does not attempt to limit or alter the - recipient's rights in the Source Code version from the rights set forth in this License. If - You distribute the Executable version 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 any 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.7. Larger Works.

    -

    You may create a Larger Work by combining Covered Code 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 Code. -

    4. Inability to Comply Due to Statute or Regulation.

    -

    If it is impossible for You to comply with any of the terms of this License with respect to - some or all of the Covered Code due to statute, judicial order, or regulation then You must: - (a) comply with the terms of this License to the maximum extent possible; and (b) describe - the limitations and the code they affect. Such description must be included in the - legal file described in Section - 3.4 and must be included with all distributions of the Source Code. - Except to the extent prohibited by statute or regulation, such description must be - sufficiently detailed for a recipient of ordinary skill to be able to understand it. -

    5. Application of this License.

    -

    This License applies to code to which the Initial Developer has attached the notice in - Exhibit A and to related Covered Code. -

    6. Versions of the License.

    -

    6.1. New Versions

    -

    Netscape Communications Corporation ("Netscape") may publish revised and/or new versions - of the License from time to time. Each version will be given a distinguishing version number. -

    6.2. Effect of New Versions

    -

    Once Covered Code has been published under a particular version of the License, You may - always continue to use it under the terms of that version. You may also choose to use such - Covered Code under the terms of any subsequent version of the License published by Netscape. - No one other than Netscape has the right to modify the terms applicable to Covered Code - created under this License. -

    6.3. Derivative Works

    -

    If You create or use a modified version of this License (which you may only do in order to - apply it to code which is not already Covered Code governed by this License), You must (a) - rename Your license so that the phrases "Mozilla", "MOZILLAPL", "MOZPL", "Netscape", "MPL", - "NPL" or any confusingly similar phrase do not appear in your license (except to note that - your license differs from this License) and (b) otherwise make it clear that Your version of - the license contains terms which differ from the Mozilla Public License and Netscape Public - License. (Filling in the name of the Initial Developer, Original Code or Contributor in the - notice described in Exhibit A shall not of themselves be deemed to - be modifications of this License.) -

    7. Disclaimer of warranty

    -

    Covered code 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 code 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 code is with you. Should any covered code 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 code is authorized hereunder except under this - disclaimer. -

    8. Termination

    -

    8.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. All sublicenses to the Covered Code which - are properly granted shall survive any termination of this License. Provisions which, by - their nature, must remain in effect beyond the termination of this License shall survive. -

    8.2. If You initiate litigation by asserting a patent infringement - claim (excluding declatory judgment actions) against Initial Developer or a Contributor - (the Initial Developer or Contributor against whom You file such action is referred to - as "Participant") alleging that: -

      -
    1. such Participant's Contributor Version directly or indirectly - infringes any patent, then any and all rights granted by such Participant to You under - Sections 2.1 and/or 2.2 of this - License shall, upon 60 days notice from Participant terminate prospectively, unless if - within 60 days after receipt of notice You either: (i) agree in writing to pay - Participant a mutually agreeable reasonable royalty for Your past and future use of - Modifications made by such Participant, or (ii) withdraw Your litigation claim with - respect to the Contributor Version against such Participant. If within 60 days of - notice, a reasonable royalty and payment arrangement are not mutually agreed upon in - writing by the parties or the litigation claim is not withdrawn, the rights granted by - Participant to You under Sections 2.1 and/or - 2.2 automatically terminate at the expiration of the 60 day - notice period specified above. -
    2. any software, hardware, or device, other than such Participant's - Contributor Version, directly or indirectly infringes any patent, then any rights - granted to You by such Participant under Sections 2.1(b) - and 2.2(b) are revoked effective as of the date You first - made, used, sold, distributed, or had made, Modifications made by that Participant. -
    -

    8.3. If You assert a patent infringement claim against Participant - alleging that such Participant's Contributor Version 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. -

    8.4. In the event of termination under Sections - 8.1 or 8.2 above, all end user - license agreements (excluding distributors and resellers) which have been validly - granted by You or any distributor hereunder prior to termination shall survive - termination. -

    9. 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 code, 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. -

    10. U.S. government end users

    -

    The Covered Code is a "commercial item," as that term is defined in 48 - C.F.R. 2.101 (Oct. 1995), consisting of - "commercial computer software" 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 Code with only those rights set forth herein. -

    11. 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 California law provisions (except to the extent applicable law, if any, - provides otherwise), excluding its conflict-of-law provisions. With respect to - disputes in which at least one party is a citizen of, or an entity chartered or - registered to do business in the United States of America, any litigation relating to - this License shall be subject to the jurisdiction of the Federal Courts of the - Northern District of California, with venue lying in Santa Clara County, California, - 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. -

    12. 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. -

    13. Multiple-licensed code

    -

    Initial Developer may designate portions of the Covered Code as - "Multiple-Licensed". "Multiple-Licensed" means that the Initial Developer permits - you to utilize portions of the Covered Code under Your choice of the MPL - or the alternative licenses, if any, specified by the Initial Developer in the file - described in Exhibit A. -

    Exhibit A - Mozilla Public License.

    -
    "The contents of this file are subject to the Mozilla Public License
    -Version 1.1 (the "License"); you may not use this file except in
    -compliance with the License. You may obtain a copy of the License at
    -http://www.mozilla.org/MPL/
    -
    -Software distributed under the License is distributed on an "AS IS"
    -basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See the
    -License for the specific language governing rights and limitations
    -under the License.
    -
    -The Original Code is JTransforms.
    -
    -The Initial Developer of the Original Code is
    -Piotr Wendykier, Emory University.
    -Portions created by the Initial Developer are Copyright (C) 2007-2009
    -the Initial Developer. All Rights Reserved.
    -
    -Alternatively, the contents of this file may be used under the terms of
    -either the GNU General Public License Version 2 or later (the "GPL"), or
    -the GNU Lesser General Public License Version 2.1 or later (the "LGPL"),
    -in which case the provisions of the GPL or the LGPL are applicable instead
    -of those above. If you wish to allow use of your version of this file only
    -under the terms of either the GPL or the LGPL, and not to allow others to
    -use your version of this file under the terms of the MPL, indicate your
    -decision by deleting the provisions above and replace them with the notice
    -and other provisions required by the GPL or the LGPL. If you do not delete
    -the provisions above, a recipient may use your version of this file under
    -the terms of any one of the MPL, the GPL or the LGPL.
    -

    NOTE: The text of this Exhibit A may differ slightly from the text of - the notices in the Source Code files of the Original Code. You should - use the text of this Exhibit A rather than the text found in the - Original Code Source Code for Your Modifications. - -

    \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 32d98151bdcff..84d6a536ccca8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import edu.emory.mathcs.jtransforms.dct._ +import org.jtransforms.dct._ import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index b7956b6fd3e9a..69952f0b64ac2 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -20,7 +20,7 @@ import java.util.Arrays; import java.util.List; -import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; +import org.jtransforms.dct.DoubleDCT_1D; import org.junit.Assert; import org.junit.Test; diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index 985e396000d05..079dabb3665be 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D +import org.jtransforms.dct.DoubleDCT_1D import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} diff --git a/pom.xml b/pom.xml index 2f29ad07f1d90..2396c5168b166 100644 --- a/pom.xml +++ b/pom.xml @@ -786,14 +786,8 @@ org.scalanlp breeze_${scala.binary.version} - 0.13.2 + 1.0 - - - junit - junit - org.apache.commons commons-math3 From f7cc6958084111cc6180c572519082ec235eb189 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 21 Sep 2019 16:29:23 +0900 Subject: [PATCH 73/87] [SPARK-29140][SQL] Handle parameters having "array" of javaType properly in splitAggregateExpressions ### What changes were proposed in this pull request? This patch fixes the issue brought by [SPARK-21870](http://issues.apache.org/jira/browse/SPARK-21870): when generating code for parameter type, it doesn't consider array type in javaType. At least we have one, Spark should generate code for BinaryType as `byte[]`, but Spark create the code for BinaryType as `[B` and generated code fails compilation. Below is the generated code which failed compilation (Line 380): ``` /* 380 */ private void agg_doAggregate_count_0([B agg_expr_1_1, boolean agg_exprIsNull_1_1, org.apache.spark.sql.catalyst.InternalRow agg_unsafeRowAggBuffer_1) throws java.io.IOException { /* 381 */ // evaluate aggregate function for count /* 382 */ boolean agg_isNull_26 = false; /* 383 */ long agg_value_28 = -1L; /* 384 */ if (!false && agg_exprIsNull_1_1) { /* 385 */ long agg_value_31 = agg_unsafeRowAggBuffer_1.getLong(1); /* 386 */ agg_isNull_26 = false; /* 387 */ agg_value_28 = agg_value_31; /* 388 */ } else { /* 389 */ long agg_value_33 = agg_unsafeRowAggBuffer_1.getLong(1); /* 390 */ /* 391 */ long agg_value_32 = -1L; /* 392 */ /* 393 */ agg_value_32 = agg_value_33 + 1L; /* 394 */ agg_isNull_26 = false; /* 395 */ agg_value_28 = agg_value_32; /* 396 */ } /* 397 */ // update unsafe row buffer /* 398 */ agg_unsafeRowAggBuffer_1.setLong(1, agg_value_28); /* 399 */ } ``` There wasn't any test for HashAggregateExec specifically testing this, but randomized test in ObjectHashAggregateSuite could encounter this and that's why ObjectHashAggregateSuite is flaky. ### Why are the changes needed? Without the fix, generated code from HashAggregateExec may fail compilation. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added new UT. Without the fix, newly added UT fails. Closes #25830 from HeartSaVioR/SPARK-29140. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Takeshi Yamamuro --- .../expressions/codegen/CodeGenerator.scala | 8 ++++++++ .../execution/aggregate/HashAggregateExec.scala | 4 +++- .../hive/execution/AggregationQuerySuite.scala | 15 +++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 68ddec9fc8d00..946fc7f421ad6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1811,6 +1811,14 @@ object CodeGenerator extends Logging { def boxedType(dt: DataType): String = boxedType(javaType(dt)) + def typeName(clazz: Class[_]): String = { + if (clazz.isArray) { + typeName(clazz.getComponentType) + "[]" + } else { + clazz.getName + } + } + /** * Returns the representation of default value for a given Java Type. * @param jt the string name of the Java type diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 2d187e3c9ebe5..5dc5b822919be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -299,7 +299,9 @@ case class HashAggregateExec( if (inputVars.forall(_.isDefined)) { val splitCodes = inputVars.flatten.zipWithIndex.map { case (args, i) => val doAggFunc = ctx.freshName(s"doAggregate_${aggNames(i)}") - val argList = args.map(v => s"${v.javaType.getName} ${v.variableName}").mkString(", ") + val argList = args.map { v => + s"${CodeGenerator.typeName(v.javaType)} ${v.variableName}" + }.mkString(", ") val doAggFuncName = ctx.addNewFunction(doAggFunc, s""" |private void $doAggFunc($argList) throws java.io.IOException { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 8c7e5bf5ac1d4..4a3277f5a7e49 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -1028,6 +1028,21 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } } } + + test("SPARK-29140: HashAggregateExec aggregating binary type doesn't break codegen compilation") { + val schema = new StructType().add("id", IntegerType, nullable = false) + .add("c1", BinaryType, nullable = true) + + withSQLConf( + SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "true", + SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1") { + val emptyRows = spark.sparkContext.parallelize(Seq.empty[Row], 1) + val aggDf = spark.createDataFrame(emptyRows, schema) + .groupBy($"id" % 10 as "group") + .agg(countDistinct($"c1")) + checkAnswer(aggDf, Seq.empty[Row]) + } + } } From 076186e8815ba6381f14941e05a6ec15348369ba Mon Sep 17 00:00:00 2001 From: colinma Date: Sat, 21 Sep 2019 07:31:39 -0500 Subject: [PATCH 74/87] [SPARK-19147][CORE] Gracefully handle error in task after executor is stopped ### What changes were proposed in this pull request? TransportClientFactory.createClient() is called by task and TransportClientFactory.close() is called by executor. When stop the executor, close() will set workerGroup = null, NPE will occur in createClient which generate many exception in log. For exception occurs after close(), treated it as an expected Exception and transform it to InterruptedException which can be processed by Executor. ### Why are the changes needed? The change can reduce the exception stack trace in log file, and user won't be confused by these excepted exception. ### Does this PR introduce any user-facing change? N/A ### How was this patch tested? New tests are added in TransportClientFactorySuite and ExecutorSuite Closes #25759 from colinmjj/spark-19147. Authored-by: colinma Signed-off-by: Sean Owen --- .../spark/network/client/TransportClientFactory.java | 3 +-- .../apache/spark/network/TransportClientFactorySuite.java | 7 +++++++ core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../main/scala/org/apache/spark/executor/Executor.scala | 5 +++++ 4 files changed, 14 insertions(+), 3 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 53835d8304866..c9ef9f918ffd1 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -293,9 +293,8 @@ public void close() { } connectionPool.clear(); - if (workerGroup != null) { + if (workerGroup != null && !workerGroup.isShuttingDown()) { workerGroup.shutdownGracefully(); - workerGroup = null; } } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 2aec4a33bbe43..9b76981c31c57 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -217,4 +217,11 @@ public Iterable> getAll() { assertFalse(c1.isActive()); } } + + @Test(expected = IOException.class) + public void closeFactoryBeforeCreateClient() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + factory.close(); + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 419f0ab065150..14895b39b24bc 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -70,7 +70,7 @@ class SparkEnv ( val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { - private[spark] var isStopped = false + @volatile private[spark] var isStopped = false private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 12bc227b341a3..d2eb01d20e9a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -629,6 +629,11 @@ private[spark] class Executor( setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(reason)) + case t: Throwable if env.isStopped => + // Log the expected exception after executor.stop without stack traces + // see: SPARK-19147 + logError(s"Exception in $taskName (TID $taskId): ${t.getMessage}") + case t: Throwable => // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to From 93ac4e1b2df5ca1dde29010320c6faead537a7b1 Mon Sep 17 00:00:00 2001 From: aman_omer Date: Sat, 21 Sep 2019 07:34:04 -0500 Subject: [PATCH 75/87] [SPARK-29053][WEBUI] Sort does not work on some columns ### What changes were proposed in this pull request? Setting custom sort key for duration and execution time column. ### Why are the changes needed? Sorting on duration and execution time columns consider time as a string after converting into readable form which is the reason for wrong sort results as mentioned in [SPARK-29053](https://issues.apache.org/jira/browse/SPARK-29053). ### Does this PR introduce any user-facing change? No ### How was this patch tested? Test manually. Screenshots are attached. After patch: **Duration** ![Duration](https://user-images.githubusercontent.com/40591404/65339861-93cc9800-dbea-11e9-95e6-63b107a5a372.png) **Execution time** ![Execution Time](https://user-images.githubusercontent.com/40591404/65339870-97601f00-dbea-11e9-9d1d-690c59bc1bde.png) Closes #25855 from amanomer/SPARK29053. Authored-by: aman_omer Signed-off-by: Sean Owen --- .../sql/hive/thriftserver/ui/ThriftServerPage.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 261e8fc912eb9..1a7db1b58e5bd 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 @@ -91,8 +91,10 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {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)))} + + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} @@ -157,7 +159,8 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {session.sessionId} {formatDate(session.startTimestamp)} {if (session.finishTimestamp > 0) formatDate(session.finishTimestamp)} - {formatDurationOption(Some(session.totalTime))} + + {formatDurationOption(Some(session.totalTime))} {session.totalExecution.toString} } From 81b6f11a3a4f0f3e0fe807646618220c0c71f869 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 21 Sep 2019 23:59:37 +0900 Subject: [PATCH 76/87] [SPARK-29160][CORE] Use UTF-8 explicitly for reading/writing event log file ### What changes were proposed in this pull request? Credit to vanzin as he found and commented on this while reviewing #25670 - [comment](https://github.com/apache/spark/pull/25670#discussion_r325383512). This patch proposes to specify UTF-8 explicitly while reading/writer event log file. ### Why are the changes needed? The event log file is being read/written as default character set of JVM process which may open the chance to bring some problems on reading event log files from another machines. Spark's de facto standard character set is UTF-8, so it should be explicitly set to. ### Does this PR introduce any user-facing change? Yes, if end users have been running Spark process with different default charset than "UTF-8", especially their driver JVM processes. No otherwise. ### How was this patch tested? Existing UTs, as ReplayListenerSuite contains "end-to-end" event logging/reading tests (both uncompressed/compressed). Closes #25845 from HeartSaVioR/SPARK-29160. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../apache/spark/scheduler/EventLoggingListener.scala | 2 +- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 4 ++-- .../apache/spark/scheduler/ReplayListenerSuite.scala | 10 +++++++--- docs/core-migration-guide.md | 1 + 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index bee784dcdfe33..3e8318e1921cb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -131,7 +131,7 @@ private[spark] class EventLoggingListener( EventLoggingListener.initEventLog(bstream, testing, loggedEvents) fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) - writer = Some(new PrintWriter(bstream)) + writer = Some(new PrintWriter(new OutputStreamWriter(bstream, StandardCharsets.UTF_8))) logInfo("Logging events to %s".format(logPath)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 226c23733c870..699042dd967bc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.io.{EOFException, InputStream, IOException} -import scala.io.Source +import scala.io.{Codec, Source} import com.fasterxml.jackson.core.JsonParseException import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException @@ -54,7 +54,7 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { sourceName: String, maybeTruncated: Boolean = false, eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { - val lines = Source.fromInputStream(logData).getLines() + val lines = Source.fromInputStream(logData)(Codec.UTF8).getLines() replay(lines, sourceName, maybeTruncated, eventsFilter) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d65b5cbfc094e..cda537159a573 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI +import java.nio.charset.StandardCharsets import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer @@ -52,10 +53,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp test("Simple replay") { val logFilePath = getFilePath(testDir, "events.txt") val fstream = fileSystem.create(logFilePath) + val fwriter = new OutputStreamWriter(fstream, StandardCharsets.UTF_8) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) - Utils.tryWithResource(new PrintWriter(fstream)) { writer => + Utils.tryWithResource(new PrintWriter(fwriter)) { writer => // scalastyle:off println writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) @@ -88,7 +90,8 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val buffered = new ByteArrayOutputStream val codec = new LZ4CompressionCodec(new SparkConf()) val compstream = codec.compressedOutputStream(buffered) - Utils.tryWithResource(new PrintWriter(compstream)) { writer => + val cwriter = new OutputStreamWriter(compstream, StandardCharsets.UTF_8) + Utils.tryWithResource(new PrintWriter(cwriter)) { writer => val applicationStart = SparkListenerApplicationStart("AppStarts", None, 125L, "Mickey", None) @@ -134,10 +137,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp test("Replay incompatible event log") { val logFilePath = getFilePath(testDir, "incompatible.txt") val fstream = fileSystem.create(logFilePath) + val fwriter = new OutputStreamWriter(fstream, StandardCharsets.UTF_8) val applicationStart = SparkListenerApplicationStart("Incompatible App", None, 125L, "UserUsingIncompatibleVersion", None) val applicationEnd = SparkListenerApplicationEnd(1000L) - Utils.tryWithResource(new PrintWriter(fstream)) { writer => + Utils.tryWithResource(new PrintWriter(fwriter)) { writer => // scalastyle:off println writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println("""{"Event":"UnrecognizedEventOnlyForTest","Timestamp":1477593059313}""") diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 1c47dfb27124c..2d4d91dab075e 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -30,3 +30,4 @@ license: | - In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. +- In Spark 3.0, event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. \ No newline at end of file From 3e2649287d98b9692fdd1025e1bf8fe8d3de768e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 21 Sep 2019 08:13:00 -0700 Subject: [PATCH 77/87] [SPARK-29199][INFRA] Add linters and license/dependency checkers to GitHub Action ### What changes were proposed in this pull request? This PR aims to add linters and license/dependency checkers to GitHub Action. This excludes `lint-r` intentionally because https://github.com/actions/setup-r is not ready. We can add that later when it becomes available. ### Why are the changes needed? This will help the PR reviews. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? See the GitHub Action result on this PR. Closes #25879 from dongjoon-hyun/SPARK-29199. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 350f3405bc4c7..e2eb0683b6e59 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -32,3 +32,29 @@ jobs: export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} package + + + lint: + runs-on: ubuntu-latest + name: Linters + steps: + - uses: actions/checkout@master + - uses: actions/setup-java@v1 + with: + java-version: '11' + - uses: actions/setup-python@v1 + with: + python-version: '3.x' + architecture: 'x64' + - name: Scala + run: ./dev/lint-scala + - name: Java + run: ./dev/lint-java + - name: Python + run: | + pip install flake8 sphinx numpy + ./dev/lint-python + - name: License + run: ./dev/check-license + - name: Dependencies + run: ./dev/test-dependencies.sh From c7c6b642dccb3fbd5f82b81cff7f88a776976c7d Mon Sep 17 00:00:00 2001 From: Patrick Pisciuneri Date: Sat, 21 Sep 2019 14:26:54 -0500 Subject: [PATCH 78/87] [SPARK-29121][ML][MLLIB] Support for dot product operation on Vector(s) ### What changes were proposed in this pull request? Support for dot product with: - `ml.linalg.Vector` - `ml.linalg.Vectors` - `mllib.linalg.Vector` - `mllib.linalg.Vectors` ### Why are the changes needed? Dot product is useful for feature engineering and scoring. BLAS routines are already there, just a wrapper is needed. ### Does this PR introduce any user-facing change? No user facing changes, just some new functionality. ### How was this patch tested? Tests were written and added to the appropriate `VectorSuites` classes. They can be quickly run with: ``` sbt "mllib-local/testOnly org.apache.spark.ml.linalg.VectorsSuite" sbt "mllib/testOnly org.apache.spark.mllib.linalg.VectorsSuite" ``` Closes #25818 from phpisciuneri/SPARK-29121. Authored-by: Patrick Pisciuneri Signed-off-by: Sean Owen --- .../org/apache/spark/ml/linalg/Vectors.scala | 8 +++++++ .../apache/spark/ml/linalg/VectorsSuite.scala | 23 +++++++++++++++++++ .../apache/spark/mllib/linalg/Vectors.scala | 8 +++++++ .../spark/mllib/linalg/VectorsSuite.scala | 23 +++++++++++++++++++ 4 files changed, 62 insertions(+) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index 6e43d60bd03a3..f437d66cddb54 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -178,6 +178,14 @@ sealed trait Vector extends Serializable { */ @Since("2.0.0") def argmax: Int + + /** + * Calculate the dot product of this vector with another. + * + * If `size` does not match an [[IllegalArgumentException]] is thrown. + */ + @Since("3.0.0") + def dot(v: Vector): Double = BLAS.dot(this, v) } /** diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala index 0a316f57f811b..c97dc2c3c06f8 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala @@ -380,4 +380,27 @@ class VectorsSuite extends SparkMLFunSuite { Vectors.sparse(-1, Array((1, 2.0))) } } + + test("dot product only supports vectors of same size") { + val vSize4 = Vectors.dense(arr) + val vSize1 = Vectors.zeros(1) + intercept[IllegalArgumentException]{ vSize1.dot(vSize4) } + } + + test("dense vector dot product") { + val dv = Vectors.dense(arr) + assert(dv.dot(dv) === 0.26) + } + + test("sparse vector dot product") { + val sv = Vectors.sparse(n, indices, values) + assert(sv.dot(sv) === 0.26) + } + + test("mixed sparse and dense vector dot product") { + val sv = Vectors.sparse(n, indices, values) + val dv = Vectors.dense(arr) + assert(sv.dot(dv) === 0.26) + assert(dv.dot(sv) === 0.26) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index b754fad0c1796..83a519326df75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -204,6 +204,14 @@ sealed trait Vector extends Serializable { */ @Since("2.0.0") def asML: newlinalg.Vector + + /** + * Calculate the dot product of this vector with another. + * + * If `size` does not match an [[IllegalArgumentException]] is thrown. + */ + @Since("3.0.0") + def dot(v: Vector): Double = BLAS.dot(this, v) } /** 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 fee0b02bf8ed8..b2163b518dbd1 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 @@ -510,4 +510,27 @@ class VectorsSuite extends SparkFunSuite with Logging { Vectors.sparse(-1, Array((1, 2.0))) } } + + test("dot product only supports vectors of same size") { + val vSize4 = Vectors.dense(arr) + val vSize1 = Vectors.zeros(1) + intercept[IllegalArgumentException]{ vSize1.dot(vSize4) } + } + + test("dense vector dot product") { + val dv = Vectors.dense(arr) + assert(dv.dot(dv) === 0.26) + } + + test("sparse vector dot product") { + val sv = Vectors.sparse(n, indices, values) + assert(sv.dot(sv) === 0.26) + } + + test("mixed sparse and dense vector dot product") { + val sv = Vectors.sparse(n, indices, values) + val dv = Vectors.dense(arr) + assert(sv.dot(dv) === 0.26) + assert(dv.dot(sv) === 0.26) + } } From 3be5741029c9ad60b56686eb6a33dce8eb8189d8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 21 Sep 2019 21:11:31 -0700 Subject: [PATCH 79/87] [SPARK-29190][SQL] Optimize `extract`/`date_part` for the milliseconds `field` ### What changes were proposed in this pull request? Changed the `DateTimeUtils.getMilliseconds()` by avoiding the decimal division, and replacing it by setting scale and precision while converting microseconds to the decimal type. ### Why are the changes needed? This improves performance of `extract` and `date_part()` by more than **50 times**: Before: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 397 428 45 25.2 39.7 1.0X MILLISECONDS of timestamp 36723 36761 63 0.3 3672.3 0.0X ``` After: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 278 284 6 36.0 27.8 1.0X MILLISECONDS of timestamp 592 606 13 16.9 59.2 0.5X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suite - `DateExpressionsSuite` Closes #25871 from MaxGekk/optimize-epoch-millis. Lead-authored-by: Maxim Gekk Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/util/DateTimeUtils.scala | 3 +- .../ExtractBenchmark-jdk11-results.txt | 100 +++++++++++ .../benchmarks/ExtractBenchmark-results.txt | 160 +++++++++--------- 3 files changed, 181 insertions(+), 82 deletions(-) create mode 100644 sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt 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 a82471aae652d..7757ea51536c9 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 @@ -465,8 +465,7 @@ object DateTimeUtils { * is expressed in microseconds since the epoch. */ def getMilliseconds(timestamp: SQLTimestamp, timeZone: TimeZone): Decimal = { - val micros = Decimal(getMicroseconds(timestamp, timeZone)) - (micros / Decimal(MICROS_PER_MILLIS)).toPrecision(8, 3) + Decimal(getMicroseconds(timestamp, timeZone), 8, 3) } /** diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..9ee9176bea508 --- /dev/null +++ b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt @@ -0,0 +1,100 @@ +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 + +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 + +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 + +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 + diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 6c4bdfed703ff..4894db4c522f6 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -2,99 +2,99 @@ 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 397 428 45 25.2 39.7 1.0X -MILLENNIUM of timestamp 1480 1546 67 6.8 148.0 0.3X -CENTURY of timestamp 1368 1384 17 7.3 136.8 0.3X -DECADE of timestamp 1281 1344 57 7.8 128.1 0.3X -YEAR of timestamp 1238 1244 5 8.1 123.8 0.3X -ISOYEAR of timestamp 1379 1455 122 7.2 137.9 0.3X -QUARTER of timestamp 1442 1456 15 6.9 144.2 0.3X -MONTH of timestamp 1213 1217 3 8.2 121.3 0.3X -WEEK of timestamp 1927 1947 22 5.2 192.7 0.2X -DAY of timestamp 1306 1320 16 7.7 130.6 0.3X -DAYOFWEEK of timestamp 1394 1402 11 7.2 139.4 0.3X -DOW of timestamp 1367 1374 6 7.3 136.7 0.3X -ISODOW of timestamp 1317 1321 5 7.6 131.7 0.3X -DOY of timestamp 1223 1238 14 8.2 122.3 0.3X -HOUR of timestamp 361 362 2 27.7 36.1 1.1X -MINUTE of timestamp 354 362 10 28.3 35.4 1.1X -SECOND of timestamp 362 365 4 27.6 36.2 1.1X -MILLISECONDS of timestamp 36723 36761 63 0.3 3672.3 0.0X -MICROSECONDS of timestamp 469 490 29 21.3 46.9 0.8X -EPOCH of timestamp 30137 30181 38 0.3 3013.7 0.0X +cast to timestamp 375 411 52 26.7 37.5 1.0X +MILLENNIUM of timestamp 1389 1410 34 7.2 138.9 0.3X +CENTURY of timestamp 1327 1345 25 7.5 132.7 0.3X +DECADE of timestamp 1214 1257 59 8.2 121.4 0.3X +YEAR of timestamp 1185 1192 6 8.4 118.5 0.3X +ISOYEAR of timestamp 1297 1371 93 7.7 129.7 0.3X +QUARTER of timestamp 1375 1395 32 7.3 137.5 0.3X +MONTH of timestamp 1179 1191 17 8.5 117.9 0.3X +WEEK of timestamp 1760 1778 21 5.7 176.0 0.2X +DAY of timestamp 1177 1185 8 8.5 117.7 0.3X +DAYOFWEEK of timestamp 1330 1331 1 7.5 133.0 0.3X +DOW of timestamp 1335 1362 43 7.5 133.5 0.3X +ISODOW of timestamp 1277 1282 8 7.8 127.7 0.3X +DOY of timestamp 1195 1208 16 8.4 119.5 0.3X +HOUR of timestamp 335 342 6 29.8 33.5 1.1X +MINUTE of timestamp 322 330 7 31.1 32.2 1.2X +SECOND of timestamp 324 330 8 30.9 32.4 1.2X +MILLISECONDS of timestamp 543 550 6 18.4 54.3 0.7X +MICROSECONDS of timestamp 426 431 8 23.5 42.6 0.9X +EPOCH of timestamp 29807 29831 29 0.3 2980.7 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 Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1010 1022 11 9.9 101.0 1.0X -MILLENNIUM of date 1300 1311 18 7.7 130.0 0.8X -CENTURY of date 1304 1306 2 7.7 130.4 0.8X -DECADE of date 1199 1205 10 8.3 119.9 0.8X -YEAR of date 1191 1194 4 8.4 119.1 0.8X -ISOYEAR of date 1451 1456 9 6.9 145.1 0.7X -QUARTER of date 1494 1501 10 6.7 149.4 0.7X -MONTH of date 1189 1191 3 8.4 118.9 0.8X -WEEK of date 1893 1958 111 5.3 189.3 0.5X -DAY of date 1282 1285 3 7.8 128.2 0.8X -DAYOFWEEK of date 1374 1386 17 7.3 137.4 0.7X -DOW of date 1348 1351 3 7.4 134.8 0.7X -ISODOW of date 1292 1297 5 7.7 129.2 0.8X -DOY of date 1213 1216 3 8.2 121.3 0.8X -HOUR of date 1450 1458 9 6.9 145.0 0.7X -MINUTE of date 1445 1452 9 6.9 144.5 0.7X -SECOND of date 1448 1458 8 6.9 144.8 0.7X -MILLISECONDS of date 2094 2103 11 4.8 209.4 0.5X -MICROSECONDS of date 1562 1573 19 6.4 156.2 0.6X -EPOCH of date 31000 31047 68 0.3 3100.0 0.0X +cast to date 976 983 7 10.2 97.6 1.0X +MILLENNIUM of date 1265 1271 7 7.9 126.5 0.8X +CENTURY of date 1273 1285 20 7.9 127.3 0.8X +DECADE of date 1158 1166 8 8.6 115.8 0.8X +YEAR of date 1165 1177 19 8.6 116.5 0.8X +ISOYEAR of date 1394 1399 5 7.2 139.4 0.7X +QUARTER of date 1463 1464 1 6.8 146.3 0.7X +MONTH of date 1153 1156 3 8.7 115.3 0.8X +WEEK of date 1743 1748 9 5.7 174.3 0.6X +DAY of date 1145 1150 5 8.7 114.5 0.9X +DAYOFWEEK of date 1315 1316 2 7.6 131.5 0.7X +DOW of date 1315 1325 14 7.6 131.5 0.7X +ISODOW of date 1267 1269 2 7.9 126.7 0.8X +DOY of date 1193 1203 17 8.4 119.3 0.8X +HOUR of date 1419 1428 15 7.0 141.9 0.7X +MINUTE of date 1416 1423 6 7.1 141.6 0.7X +SECOND of date 1412 1416 6 7.1 141.2 0.7X +MILLISECONDS of date 1671 1676 9 6.0 167.1 0.6X +MICROSECONDS of date 1524 1528 4 6.6 152.4 0.6X +EPOCH of date 30922 30959 54 0.3 3092.2 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 Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 327 333 10 30.6 32.7 1.0X -MILLENNIUM of timestamp 1292 1296 4 7.7 129.2 0.3X -CENTURY of timestamp 1301 1305 6 7.7 130.1 0.3X -DECADE of timestamp 1200 1204 6 8.3 120.0 0.3X -YEAR of timestamp 1185 1193 8 8.4 118.5 0.3X -ISOYEAR of timestamp 1449 1469 18 6.9 144.9 0.2X -QUARTER of timestamp 1497 1505 7 6.7 149.7 0.2X -MONTH of timestamp 1185 1188 3 8.4 118.5 0.3X -WEEK of timestamp 1901 1909 7 5.3 190.1 0.2X -DAY of timestamp 1278 1282 4 7.8 127.8 0.3X -DAYOFWEEK of timestamp 1371 1376 5 7.3 137.1 0.2X -DOW of timestamp 1361 1372 17 7.3 136.1 0.2X -ISODOW of timestamp 1299 1306 9 7.7 129.9 0.3X -DOY of timestamp 1216 1219 4 8.2 121.6 0.3X -HOUR of timestamp 352 356 5 28.4 35.2 0.9X -MINUTE of timestamp 350 369 17 28.6 35.0 0.9X -SECOND of timestamp 351 364 19 28.5 35.1 0.9X -MILLISECONDS of timestamp 36989 37022 52 0.3 3698.9 0.0X -MICROSECONDS of timestamp 473 476 2 21.1 47.3 0.7X -EPOCH of timestamp 29890 29908 27 0.3 2989.0 0.0X +cast to timestamp 292 296 7 34.3 29.2 1.0X +MILLENNIUM of timestamp 1263 1274 10 7.9 126.3 0.2X +CENTURY of timestamp 1271 1275 4 7.9 127.1 0.2X +DECADE of timestamp 1154 1157 3 8.7 115.4 0.3X +YEAR of timestamp 1151 1157 10 8.7 115.1 0.3X +ISOYEAR of timestamp 1392 1393 1 7.2 139.2 0.2X +QUARTER of timestamp 1463 1476 12 6.8 146.3 0.2X +MONTH of timestamp 1157 1173 20 8.6 115.7 0.3X +WEEK of timestamp 1742 1749 9 5.7 174.2 0.2X +DAY of timestamp 1140 1145 5 8.8 114.0 0.3X +DAYOFWEEK of timestamp 1312 1317 5 7.6 131.2 0.2X +DOW of timestamp 1318 1321 3 7.6 131.8 0.2X +ISODOW of timestamp 1268 1270 2 7.9 126.8 0.2X +DOY of timestamp 1194 1197 3 8.4 119.4 0.2X +HOUR of timestamp 327 330 4 30.6 32.7 0.9X +MINUTE of timestamp 320 326 9 31.3 32.0 0.9X +SECOND of timestamp 320 329 16 31.2 32.0 0.9X +MILLISECONDS of timestamp 540 544 7 18.5 54.0 0.5X +MICROSECONDS of timestamp 431 438 11 23.2 43.1 0.7X +EPOCH of timestamp 29802 29824 32 0.3 2980.2 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 Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1005 1006 1 9.9 100.5 1.0X -MILLENNIUM of date 1295 1300 5 7.7 129.5 0.8X -CENTURY of date 1297 1298 1 7.7 129.7 0.8X -DECADE of date 1198 1208 13 8.3 119.8 0.8X -YEAR of date 1184 1193 13 8.4 118.4 0.8X -ISOYEAR of date 1445 1460 13 6.9 144.5 0.7X -QUARTER of date 1495 1500 4 6.7 149.5 0.7X -MONTH of date 1176 1179 3 8.5 117.6 0.9X -WEEK of date 1893 1904 15 5.3 189.3 0.5X -DAY of date 1275 1283 8 7.8 127.5 0.8X -DAYOFWEEK of date 1369 1373 4 7.3 136.9 0.7X -DOW of date 1353 1354 2 7.4 135.3 0.7X -ISODOW of date 1290 1290 1 7.8 129.0 0.8X -DOY of date 1208 1212 4 8.3 120.8 0.8X -HOUR of date 1446 1449 2 6.9 144.6 0.7X -MINUTE of date 1441 1442 1 6.9 144.1 0.7X -SECOND of date 1443 1450 8 6.9 144.3 0.7X -MILLISECONDS of date 2087 2089 3 4.8 208.7 0.5X -MICROSECONDS of date 1557 1570 21 6.4 155.7 0.6X -EPOCH of date 30980 31001 32 0.3 3098.0 0.0X +cast to date 976 981 8 10.2 97.6 1.0X +MILLENNIUM of date 1264 1268 7 7.9 126.4 0.8X +CENTURY of date 1267 1275 14 7.9 126.7 0.8X +DECADE of date 1150 1153 4 8.7 115.0 0.8X +YEAR of date 1150 1154 6 8.7 115.0 0.8X +ISOYEAR of date 1390 1393 3 7.2 139.0 0.7X +QUARTER of date 1459 1462 3 6.9 145.9 0.7X +MONTH of date 1149 1150 1 8.7 114.9 0.8X +WEEK of date 1740 1741 1 5.7 174.0 0.6X +DAY of date 1145 1153 14 8.7 114.5 0.9X +DAYOFWEEK of date 1311 1314 3 7.6 131.1 0.7X +DOW of date 1318 1320 3 7.6 131.8 0.7X +ISODOW of date 1264 1266 2 7.9 126.4 0.8X +DOY of date 1187 1189 3 8.4 118.7 0.8X +HOUR of date 1418 1419 1 7.1 141.8 0.7X +MINUTE of date 1410 1412 2 7.1 141.0 0.7X +SECOND of date 1414 1415 1 7.1 141.4 0.7X +MILLISECONDS of date 1669 1676 5 6.0 166.9 0.6X +MICROSECONDS of date 1519 1525 6 6.6 151.9 0.6X +EPOCH of date 30900 30950 56 0.3 3090.0 0.0X From 89bad267d4b8cf451d1906aa57764d97e1a565f4 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Sep 2019 16:59:59 +0900 Subject: [PATCH 80/87] [SPARK-29200][SQL] Optimize `extract`/`date_part` for epoch ### What changes were proposed in this pull request? Refactoring of the `DateTimeUtils.getEpoch()` function by avoiding decimal operations that are pretty expensive, and converting the final result to the decimal type at the end. ### Why are the changes needed? The changes improve performance of the `getEpoch()` method at least up to **20 times**. Before: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 256 277 33 39.0 25.6 1.0X EPOCH of timestamp 23455 23550 131 0.4 2345.5 0.0X ``` After: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 255 294 34 39.2 25.5 1.0X EPOCH of timestamp 1049 1054 9 9.5 104.9 0.2X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test from `DateExpressionSuite`. Closes #25881 from MaxGekk/optimize-extract-epoch. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/util/DateTimeUtils.scala | 7 +- .../benchmarks/ExtractBenchmark-results.txt | 176 +++++++++--------- 2 files changed, 92 insertions(+), 91 deletions(-) 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 7757ea51536c9..ffadd95052513 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 @@ -847,9 +847,10 @@ object DateTimeUtils { * since 1970-01-01 00:00:00 local time. */ def getEpoch(timestamp: SQLTimestamp, zoneId: ZoneId): Decimal = { - val offset = zoneId.getRules.getOffset(microsToInstant(timestamp)).getTotalSeconds - val sinceEpoch = BigDecimal(timestamp) / MICROS_PER_SECOND + offset - new Decimal().set(sinceEpoch, 20, 6) + val offset = SECONDS.toMicros( + zoneId.getRules.getOffset(microsToInstant(timestamp)).getTotalSeconds) + val sinceEpoch = timestamp + offset + Decimal(sinceEpoch, 20, 6) } def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 4894db4c522f6..2e1df667bbe8c 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,100 +1,100 @@ -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 +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 Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 375 411 52 26.7 37.5 1.0X -MILLENNIUM of timestamp 1389 1410 34 7.2 138.9 0.3X -CENTURY of timestamp 1327 1345 25 7.5 132.7 0.3X -DECADE of timestamp 1214 1257 59 8.2 121.4 0.3X -YEAR of timestamp 1185 1192 6 8.4 118.5 0.3X -ISOYEAR of timestamp 1297 1371 93 7.7 129.7 0.3X -QUARTER of timestamp 1375 1395 32 7.3 137.5 0.3X -MONTH of timestamp 1179 1191 17 8.5 117.9 0.3X -WEEK of timestamp 1760 1778 21 5.7 176.0 0.2X -DAY of timestamp 1177 1185 8 8.5 117.7 0.3X -DAYOFWEEK of timestamp 1330 1331 1 7.5 133.0 0.3X -DOW of timestamp 1335 1362 43 7.5 133.5 0.3X -ISODOW of timestamp 1277 1282 8 7.8 127.7 0.3X -DOY of timestamp 1195 1208 16 8.4 119.5 0.3X -HOUR of timestamp 335 342 6 29.8 33.5 1.1X -MINUTE of timestamp 322 330 7 31.1 32.2 1.2X -SECOND of timestamp 324 330 8 30.9 32.4 1.2X -MILLISECONDS of timestamp 543 550 6 18.4 54.3 0.7X -MICROSECONDS of timestamp 426 431 8 23.5 42.6 0.9X -EPOCH of timestamp 29807 29831 29 0.3 2980.7 0.0X +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 -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 +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 Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 976 983 7 10.2 97.6 1.0X -MILLENNIUM of date 1265 1271 7 7.9 126.5 0.8X -CENTURY of date 1273 1285 20 7.9 127.3 0.8X -DECADE of date 1158 1166 8 8.6 115.8 0.8X -YEAR of date 1165 1177 19 8.6 116.5 0.8X -ISOYEAR of date 1394 1399 5 7.2 139.4 0.7X -QUARTER of date 1463 1464 1 6.8 146.3 0.7X -MONTH of date 1153 1156 3 8.7 115.3 0.8X -WEEK of date 1743 1748 9 5.7 174.3 0.6X -DAY of date 1145 1150 5 8.7 114.5 0.9X -DAYOFWEEK of date 1315 1316 2 7.6 131.5 0.7X -DOW of date 1315 1325 14 7.6 131.5 0.7X -ISODOW of date 1267 1269 2 7.9 126.7 0.8X -DOY of date 1193 1203 17 8.4 119.3 0.8X -HOUR of date 1419 1428 15 7.0 141.9 0.7X -MINUTE of date 1416 1423 6 7.1 141.6 0.7X -SECOND of date 1412 1416 6 7.1 141.2 0.7X -MILLISECONDS of date 1671 1676 9 6.0 167.1 0.6X -MICROSECONDS of date 1524 1528 4 6.6 152.4 0.6X -EPOCH of date 30922 30959 54 0.3 3092.2 0.0X +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 -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 +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 Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 292 296 7 34.3 29.2 1.0X -MILLENNIUM of timestamp 1263 1274 10 7.9 126.3 0.2X -CENTURY of timestamp 1271 1275 4 7.9 127.1 0.2X -DECADE of timestamp 1154 1157 3 8.7 115.4 0.3X -YEAR of timestamp 1151 1157 10 8.7 115.1 0.3X -ISOYEAR of timestamp 1392 1393 1 7.2 139.2 0.2X -QUARTER of timestamp 1463 1476 12 6.8 146.3 0.2X -MONTH of timestamp 1157 1173 20 8.6 115.7 0.3X -WEEK of timestamp 1742 1749 9 5.7 174.2 0.2X -DAY of timestamp 1140 1145 5 8.8 114.0 0.3X -DAYOFWEEK of timestamp 1312 1317 5 7.6 131.2 0.2X -DOW of timestamp 1318 1321 3 7.6 131.8 0.2X -ISODOW of timestamp 1268 1270 2 7.9 126.8 0.2X -DOY of timestamp 1194 1197 3 8.4 119.4 0.2X -HOUR of timestamp 327 330 4 30.6 32.7 0.9X -MINUTE of timestamp 320 326 9 31.3 32.0 0.9X -SECOND of timestamp 320 329 16 31.2 32.0 0.9X -MILLISECONDS of timestamp 540 544 7 18.5 54.0 0.5X -MICROSECONDS of timestamp 431 438 11 23.2 43.1 0.7X -EPOCH of timestamp 29802 29824 32 0.3 2980.2 0.0X +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 -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 +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 Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 976 981 8 10.2 97.6 1.0X -MILLENNIUM of date 1264 1268 7 7.9 126.4 0.8X -CENTURY of date 1267 1275 14 7.9 126.7 0.8X -DECADE of date 1150 1153 4 8.7 115.0 0.8X -YEAR of date 1150 1154 6 8.7 115.0 0.8X -ISOYEAR of date 1390 1393 3 7.2 139.0 0.7X -QUARTER of date 1459 1462 3 6.9 145.9 0.7X -MONTH of date 1149 1150 1 8.7 114.9 0.8X -WEEK of date 1740 1741 1 5.7 174.0 0.6X -DAY of date 1145 1153 14 8.7 114.5 0.9X -DAYOFWEEK of date 1311 1314 3 7.6 131.1 0.7X -DOW of date 1318 1320 3 7.6 131.8 0.7X -ISODOW of date 1264 1266 2 7.9 126.4 0.8X -DOY of date 1187 1189 3 8.4 118.7 0.8X -HOUR of date 1418 1419 1 7.1 141.8 0.7X -MINUTE of date 1410 1412 2 7.1 141.0 0.7X -SECOND of date 1414 1415 1 7.1 141.4 0.7X -MILLISECONDS of date 1669 1676 5 6.0 166.9 0.6X -MICROSECONDS of date 1519 1525 6 6.6 151.9 0.6X -EPOCH of date 30900 30950 56 0.3 3090.0 0.0X +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 From e2c47876e9bd7faacb196baf517f549d58391002 Mon Sep 17 00:00:00 2001 From: madianjun Date: Sun, 22 Sep 2019 17:22:37 +0900 Subject: [PATCH 81/87] [CORE][MINOR] Correct a log message in DAGScheduler ### What changes were proposed in this pull request? Correct a word in a log message. ### Why are the changes needed? Log message will be more clearly. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Test is not needed. Closes #25880 from mdianjun/fix-a-word. Authored-by: madianjun Signed-off-by: HyukjinKwon --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 894234f70e05a..f9d2df8185ec7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1521,8 +1521,8 @@ private[spark] class DAGScheduler( markStageAsFinished(failedStage, errorMessage = Some(failureMessage), willRetry = !shouldAbortStage) } else { - logDebug(s"Received fetch failure from $task, but its from $failedStage which is no " + - s"longer running") + logDebug(s"Received fetch failure from $task, but it's from $failedStage which is no " + + "longer running") } if (mapStage.rdd.isBarrier()) { From 051e691029c456fc2db5f229485d3fb8f5d0e84c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Sep 2019 17:31:33 +0900 Subject: [PATCH 82/87] [SPARK-28141][SQL] Support special date values ### What changes were proposed in this pull request? Supported special string values for `DATE` type. They are simply notational shorthands that will be converted to ordinary date values when read. The following string values are supported: - `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: ```sql spark-sql> SELECT date 'tomorrow' - date 'yesterday'; 2 ``` ### Why are the changes needed? To maintain feature parity with PostgreSQL, see [8.5.1.4. Special Values](https://www.postgresql.org/docs/12/datatype-datetime.html) ### Does this PR introduce any user-facing change? Previously, the parser fails on the special values with the error: ```sql spark-sql> select date 'today'; Error in query: Cannot parse the DATE value: today(line 1, pos 7) ``` After the changes, the special values are converted to appropriate dates: ```sql spark-sql> select date 'today'; 2019-09-06 ``` ### How was this patch tested? - Added tests to `DateFormatterSuite` to check parsing special values from regular strings. - Tests in `DateTimeUtilsSuite` check parsing those values from `UTF8String` - Uncommented tests in `date.sql` Closes #25708 from MaxGekk/datetime-special-values. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/catalog/interface.scala | 4 +- .../sql/catalyst/csv/UnivocityGenerator.scala | 5 +- .../sql/catalyst/csv/UnivocityParser.scala | 5 +- .../spark/sql/catalyst/expressions/Cast.scala | 51 +-- .../expressions/datetimeExpressions.scala | 4 +- .../sql/catalyst/expressions/literals.scala | 4 +- .../sql/catalyst/json/JacksonGenerator.scala | 5 +- .../sql/catalyst/json/JacksonParser.scala | 5 +- .../sql/catalyst/parser/AstBuilder.scala | 3 +- .../sql/catalyst/util/DateFormatter.scala | 22 +- .../sql/catalyst/util/DateTimeUtils.scala | 31 +- .../expressions/HashExpressionsSuite.scala | 2 +- .../catalyst/util/DateTimeUtilsSuite.scala | 62 ++-- .../sql/catalyst/util/UnsafeArraySuite.scala | 6 +- .../spark/sql/util/DateFormatterSuite.scala | 31 +- .../spark/sql/execution/HiveResult.scala | 6 +- .../datasources/PartitioningUtils.scala | 4 +- .../datasources/jdbc/JDBCRelation.scala | 6 +- .../resources/sql-tests/inputs/pgSQL/date.sql | 17 +- .../sql-tests/results/pgSQL/date.sql.out | 328 +++++++++++------- .../apache/spark/sql/CsvFunctionsSuite.scala | 11 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 11 +- .../ParquetPartitionDiscoverySuite.scala | 2 +- 23 files changed, 400 insertions(+), 225 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 01b21feab0dd9..f653bf41c1624 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -491,7 +491,7 @@ object CatalogColumnStat extends Logging { dataType match { case BooleanType => s.toBoolean case DateType if version == 1 => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s)) - case DateType => DateFormatter().parse(s) + case DateType => DateFormatter(ZoneOffset.UTC).parse(s) case TimestampType if version == 1 => DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) case TimestampType => getTimestampFormatter().parse(s) @@ -516,7 +516,7 @@ object CatalogColumnStat extends Logging { */ def toExternalString(v: Any, colName: String, dataType: DataType): String = { val externalValue = dataType match { - case DateType => DateFormatter().format(v.asInstanceOf[Int]) + case DateType => DateFormatter(ZoneOffset.UTC).format(v.asInstanceOf[Int]) case TimestampType => getTimestampFormatter().format(v.asInstanceOf[Long]) case BooleanType | _: IntegralType | FloatType | DoubleType => v case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 9ca94501f5c58..05cb91d10868e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -45,7 +45,10 @@ class UnivocityGenerator( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) private def makeConverter(dataType: DataType): ValueConverter = dataType match { case DateType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 39a08ec06c6a0..661525a65294d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -78,7 +78,10 @@ class UnivocityParser( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) // Retrieve the raw record string. private def getCurrentInput: UTF8String = { 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 5001b4350d778..118f261de775d 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 @@ -106,7 +106,7 @@ object Cast { * * Cast.castToTimestamp */ def needsTimeZone(from: DataType, to: DataType): Boolean = (from, to) match { - case (StringType, TimestampType) => true + case (StringType, TimestampType | DateType) => true case (DateType, TimestampType) => true case (TimestampType, StringType) => true case (TimestampType, DateType) => true @@ -287,7 +287,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // [[func]] assumes the input is no longer null because eval already does the null check. @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) - private lazy val dateFormatter = DateFormatter() + private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) private val failOnIntegralTypeOverflow = SQLConf.get.ansiEnabled @@ -469,7 +469,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => DateTimeUtils.stringToDate(s).orNull) + buildCast[UTF8String](_, s => DateTimeUtils.stringToDate(s, zoneId).orNull) case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. @@ -1056,28 +1056,35 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToDateCode( from: DataType, - ctx: CodegenContext): CastFunction = from match { - case StringType => - val intOpt = ctx.freshVariable("intOpt", classOf[Option[Integer]]) - (c, evPrim, evNull) => code""" - scala.Option $intOpt = - org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDate($c); - if ($intOpt.isDefined()) { - $evPrim = ((Integer) $intOpt.get()).intValue(); - } else { - $evNull = true; - } - """ - case TimestampType => + ctx: CodegenContext): CastFunction = { + def getZoneId() = { val zoneIdClass = classOf[ZoneId] - val zid = JavaCode.global( + JavaCode.global( ctx.addReferenceObj("zoneId", zoneId, zoneIdClass.getName), zoneIdClass) - (c, evPrim, evNull) => - code"""$evPrim = - org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);""" - case _ => - (c, evPrim, evNull) => code"$evNull = true;" + } + from match { + case StringType => + val intOpt = ctx.freshVariable("intOpt", classOf[Option[Integer]]) + val zid = getZoneId() + (c, evPrim, evNull) => + code""" + scala.Option $intOpt = + org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDate($c, $zid); + if ($intOpt.isDefined()) { + $evPrim = ((Integer) $intOpt.get()).intValue(); + } else { + $evNull = true; + } + """ + case TimestampType => + val zid = getZoneId() + (c, evPrim, evNull) => + code"""$evPrim = + org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);""" + case _ => + (c, evPrim, evNull) => code"$evNull = true;" + } } private[this] def changePrecision(d: ExprValue, decimalType: DecimalType, 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 592b9de83d9a0..0098226b3258c 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 @@ -76,9 +76,7 @@ case class CurrentDate(timeZoneId: Option[String] = None) override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) - override def eval(input: InternalRow): Any = { - localDateToDays(LocalDate.now(zoneId)) - } + override def eval(input: InternalRow): Any = currentDate(zoneId) override def prettyName: String = "current_date" } 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 9cef3ecadc543..4793b5942a79e 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 @@ -371,7 +371,9 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case _ => v + "D" } case (v: Decimal, t: DecimalType) => v + "BD" - case (v: Int, DateType) => s"DATE '${DateFormatter().format(v)}'" + case (v: Int, DateType) => + val formatter = DateFormatter(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + s"DATE '${formatter.format(v)}'" case (v: Long, TimestampType) => val formatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) 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 3378040d1b640..3ee7e484690d5 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 @@ -81,7 +81,10 @@ private[sql] class JacksonGenerator( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) private def makeWriter(dataType: DataType): ValueWriter = dataType match { case NullType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 19bc5bf3b29e3..b534b5a3d2d64 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -59,7 +59,10 @@ class JacksonParser( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) /** * Create a converter which converts the JSON documents held by the `JsonParser` 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 c334d09b691ea..74c485e091cd8 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 @@ -1734,7 +1734,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } try { valueType match { - case "DATE" => toLiteral(stringToDate, DateType) + case "DATE" => + toLiteral(stringToDate(_, getZoneId(SQLConf.get.sessionLocalTimeZone)), DateType) case "TIMESTAMP" => val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) toLiteral(stringToTimestamp(_, zoneId), TimestampType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 4940aa83a3017..7f982b019c8d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.catalyst.util -import java.time.LocalDate +import java.time.{LocalDate, ZoneId} import java.util.Locale +import DateTimeUtils.{convertSpecialDate, localDateToDays} + sealed trait DateFormatter extends Serializable { def parse(s: String): Int // returns days since epoch def format(days: Int): String @@ -27,14 +29,18 @@ sealed trait DateFormatter extends Serializable { class Iso8601DateFormatter( pattern: String, + zoneId: ZoneId, locale: Locale) extends DateFormatter with DateTimeFormatterHelper { @transient private lazy val formatter = getOrCreateFormatter(pattern, locale) override def parse(s: String): Int = { - val localDate = LocalDate.parse(s, formatter) - DateTimeUtils.localDateToDays(localDate) + val specialDate = convertSpecialDate(s.trim, zoneId) + specialDate.getOrElse { + val localDate = LocalDate.parse(s, formatter) + localDateToDays(localDate) + } } override def format(days: Int): String = { @@ -46,11 +52,13 @@ object DateFormatter { val defaultPattern: String = "uuuu-MM-dd" val defaultLocale: Locale = Locale.US - def apply(format: String, locale: Locale): DateFormatter = { - new Iso8601DateFormatter(format, locale) + def apply(format: String, zoneId: ZoneId, locale: Locale): DateFormatter = { + new Iso8601DateFormatter(format, zoneId, locale) } - def apply(format: String): DateFormatter = apply(format, defaultLocale) + def apply(format: String, zoneId: ZoneId): DateFormatter = { + apply(format, zoneId, defaultLocale) + } - def apply(): DateFormatter = apply(defaultPattern) + def apply(zoneId: ZoneId): DateFormatter = apply(defaultPattern, zoneId) } 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 ffadd95052513..3cf310a0a9683 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 @@ -24,7 +24,6 @@ import java.time.temporal.{ChronoField, ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ -import scala.util.Try import scala.util.control.NonFatal import org.apache.spark.sql.types.Decimal @@ -379,7 +378,7 @@ object DateTimeUtils { * `yyyy-[m]m-[d]d *` * `yyyy-[m]m-[d]dT*` */ - def stringToDate(s: UTF8String): Option[SQLDate] = { + def stringToDate(s: UTF8String, zoneId: ZoneId): Option[SQLDate] = { if (s == null) { return None } @@ -387,6 +386,8 @@ object DateTimeUtils { var i = 0 var currentSegmentValue = 0 val bytes = s.trim.getBytes + val specialDate = convertSpecialDate(bytes, zoneId) + if (specialDate.isDefined) return specialDate var j = 0 while (j < bytes.length && (i < 3 && !(bytes(j) == ' ' || bytes(j) == 'T'))) { val b = bytes(j) @@ -855,6 +856,8 @@ object DateTimeUtils { def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) + def currentDate(zoneId: ZoneId): SQLDate = localDateToDays(LocalDate.now(zoneId)) + private def today(zoneId: ZoneId): ZonedDateTime = { Instant.now().atZone(zoneId).`with`(LocalTime.MIDNIGHT) } @@ -915,4 +918,28 @@ object DateTimeUtils { None } } + + /** + * Converts notational shorthands that are converted to ordinary dates. + * @param input - a trimmed string + * @param zoneId - zone identifier used to get the current date. + * @return some of days since the epoch if the conversion completed successfully otherwise None. + */ + def convertSpecialDate(input: String, zoneId: ZoneId): Option[SQLDate] = { + extractSpecialValue(input, zoneId).flatMap { + case "epoch" => Some(0) + case "now" | "today" => Some(currentDate(zoneId)) + case "tomorrow" => Some(Math.addExact(currentDate(zoneId), 1)) + case "yesterday" => Some(Math.subtractExact(currentDate(zoneId), 1)) + case _ => None + } + } + + private def convertSpecialDate(bytes: Array[Byte], zoneId: ZoneId): Option[SQLDate] = { + if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) { + convertSpecialDate(new String(bytes, StandardCharsets.UTF_8), zoneId) + } else { + None + } + } } 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 b5cfaf8f4b0fd..f90c98be0b3fd 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 @@ -174,7 +174,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for date type") { def checkHiveHashForDateType(dateString: String, expected: Long): Unit = { checkHiveHash( - DateTimeUtils.stringToDate(UTF8String.fromString(dateString)).get, + DateTimeUtils.stringToDate(UTF8String.fromString(dateString), ZoneOffset.UTC).get, DateType, expected) } 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 31fefd613f9c8..1da8efe4ef42c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{LocalDateTime, LocalTime, ZoneId} +import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit @@ -120,28 +120,32 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { checkFromToJavaDate(new Date(df2.parse("1776-07-04 18:30:00 UTC").getTime)) } + private def toDate(s: String, zoneId: ZoneId = ZoneOffset.UTC): Option[SQLDate] = { + stringToDate(UTF8String.fromString(s), zoneId) + } + test("string to date") { - assert(stringToDate(UTF8String.fromString("2015-01-28")).get === days(2015, 1, 28)) - assert(stringToDate(UTF8String.fromString("2015")).get === days(2015, 1, 1)) - assert(stringToDate(UTF8String.fromString("0001")).get === days(1, 1, 1)) - assert(stringToDate(UTF8String.fromString("2015-03")).get === days(2015, 3, 1)) + assert(toDate("2015-01-28").get === days(2015, 1, 28)) + assert(toDate("2015").get === days(2015, 1, 1)) + assert(toDate("0001").get === days(1, 1, 1)) + assert(toDate("2015-03").get === days(2015, 3, 1)) Seq("2015-03-18", "2015-03-18 ", " 2015-03-18", " 2015-03-18 ", "2015-03-18 123142", "2015-03-18T123123", "2015-03-18T").foreach { s => - assert(stringToDate(UTF8String.fromString(s)).get === days(2015, 3, 18)) + assert(toDate(s).get === days(2015, 3, 18)) } - assert(stringToDate(UTF8String.fromString("2015-03-18X")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015/03/18")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015.03.18")).isEmpty) - assert(stringToDate(UTF8String.fromString("20150318")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015-031-8")).isEmpty) - assert(stringToDate(UTF8String.fromString("02015-03-18")).isEmpty) - assert(stringToDate(UTF8String.fromString("015-03-18")).isEmpty) - assert(stringToDate(UTF8String.fromString("015")).isEmpty) - assert(stringToDate(UTF8String.fromString("02015")).isEmpty) - assert(stringToDate(UTF8String.fromString("1999 08 01")).isEmpty) - assert(stringToDate(UTF8String.fromString("1999-08 01")).isEmpty) - assert(stringToDate(UTF8String.fromString("1999 08")).isEmpty) + assert(toDate("2015-03-18X").isEmpty) + assert(toDate("2015/03/18").isEmpty) + assert(toDate("2015.03.18").isEmpty) + assert(toDate("20150318").isEmpty) + assert(toDate("2015-031-8").isEmpty) + assert(toDate("02015-03-18").isEmpty) + assert(toDate("015-03-18").isEmpty) + assert(toDate("015").isEmpty) + assert(toDate("02015").isEmpty) + assert(toDate("1999 08 01").isEmpty) + assert(toDate("1999-08 01").isEmpty) + assert(toDate("1999 08").isEmpty) } private def toTimestamp(str: String, zoneId: ZoneId): Option[SQLTimestamp] = { @@ -264,12 +268,10 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { test("SPARK-15379: special invalid date string") { // Test stringToDate - assert(stringToDate( - UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) - assert(stringToDate( - UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015-02-29")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015-04-31")).isEmpty) + assert(toDate("2015-02-29 00:00:00").isEmpty) + assert(toDate("2015-04-31 00:00:00").isEmpty) + assert(toDate("2015-02-29").isEmpty) + assert(toDate("2015-04-31").isEmpty) // Test stringToTimestamp @@ -586,4 +588,16 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { toTimestamp(" tomorrow CET ", zoneId).get should be (today + MICROS_PER_DAY +- tolerance) } } + + test("special date values") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + assert(toDate("epoch", zoneId).get === 0) + val today = localDateToDays(LocalDate.now(zoneId)) + assert(toDate("YESTERDAY", zoneId).get === today - 1) + assert(toDate(" Now ", zoneId).get === today) + assert(toDate("now UTC", zoneId) === None) // "now" does not accept time zones + assert(toDate("today", zoneId).get === today) + assert(toDate("tomorrow CET ", zoneId).get === today + 1) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 0b9e023b0b450..41adf845a6fab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import java.time.ZoneId +import java.time.{ZoneId, ZoneOffset} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -38,8 +38,8 @@ class UnsafeArraySuite extends SparkFunSuite { val doubleArray = Array(1.1, 2.2, 3.3) val stringArray = Array("1", "10", "100") val dateArray = Array( - DateTimeUtils.stringToDate(UTF8String.fromString("1970-1-1")).get, - DateTimeUtils.stringToDate(UTF8String.fromString("2016-7-26")).get) + DateTimeUtils.stringToDate(UTF8String.fromString("1970-1-1"), ZoneOffset.UTC).get, + DateTimeUtils.stringToDate(UTF8String.fromString("2016-7-26"), ZoneOffset.UTC).get) private def defaultZoneId = ZoneId.systemDefault() val timestampArray = Array( DateTimeUtils.stringToTimestamp( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala index 1f0eff2e5b114..291d40a9e84d3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.sql.util -import java.time.LocalDate +import java.time.{LocalDate, ZoneOffset} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, localDateToDays} import org.apache.spark.sql.internal.SQLConf class DateFormatterSuite extends SparkFunSuite with SQLHelper { test("parsing dates") { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val daysSinceEpoch = formatter.parse("2018-12-02") assert(daysSinceEpoch === 17867) } @@ -38,7 +39,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { test("format dates") { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val date = formatter.format(17867) assert(date === "2018-12-02") } @@ -58,7 +59,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { "5010-11-17").foreach { date => DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val days = formatter.parse(date) val formatted = formatter.format(days) assert(date === formatted) @@ -81,7 +82,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { 1110657).foreach { days => DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val date = formatter.format(days) val parsed = formatter.parse(date) assert(days === parsed) @@ -91,13 +92,29 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { } test("parsing date without explicit day") { - val formatter = DateFormatter("yyyy MMM") + val formatter = DateFormatter("yyyy MMM", ZoneOffset.UTC) val daysSinceEpoch = formatter.parse("2018 Dec") assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay) } test("formatting negative years with default pattern") { val epochDays = LocalDate.of(-99, 1, 1).toEpochDay.toInt - assert(DateFormatter().format(epochDays) === "-0099-01-01") + assert(DateFormatter(ZoneOffset.UTC).format(epochDays) === "-0099-01-01") + } + + test("special date values") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val zoneId = getZoneId(timeZone) + val formatter = DateFormatter(zoneId) + + assert(formatter.parse("EPOCH") === 0) + val today = localDateToDays(LocalDate.now(zoneId)) + assert(formatter.parse("Yesterday") === today - 1) + assert(formatter.parse("now") === today) + assert(formatter.parse("today ") === today) + assert(formatter.parse("tomorrow UTC") === today + 1) + } + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index eec8d70b5adf0..75abac4cfd1da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -77,9 +77,9 @@ object HiveResult { TimestampType, BinaryType) - private lazy val dateFormatter = DateFormatter() - private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter( - DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + private lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + private lazy val dateFormatter = DateFormatter(zoneId) + private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) /** Hive outputs fields of structs slightly differently than top level attributes. */ private def toHiveStructString(a: (Any, DataType)): String = a match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 1e47d53b7e976..fdad43b23c5aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -130,7 +130,7 @@ object PartitioningUtils { Map.empty[String, String] } - val dateFormatter = DateFormatter() + val dateFormatter = DateFormatter(zoneId) val timestampFormatter = TimestampFormatter(timestampPartitionPattern, zoneId) // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => @@ -492,7 +492,7 @@ object PartitioningUtils { // We need to check that we can cast the raw string since we later can use Cast to get // the partition values with the right DataType (see // org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning) - val dateValue = Cast(Literal(raw), DateType).eval() + val dateValue = Cast(Literal(raw), DateType, Some(zoneId.getId)).eval() // Disallow DateType if the cast returned null require(dateValue != null) Literal.create(dateValue, DateType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 3cd5cb1647923..f5a474ddf3904 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -186,7 +186,7 @@ private[sql] object JDBCRelation extends Logging { } columnType match { case _: NumericType => value.toLong - case DateType => parse(stringToDate).toLong + case DateType => parse(stringToDate(_, getZoneId(timeZoneId))).toLong case TimestampType => parse(stringToTimestamp(_, getZoneId(timeZoneId))) } } @@ -197,7 +197,9 @@ private[sql] object JDBCRelation extends Logging { timeZoneId: String): String = { def dateTimeToString(): String = { val dateTimeStr = columnType match { - case DateType => DateFormatter().format(value.toInt) + case DateType => + val dateFormatter = DateFormatter(DateTimeUtils.getZoneId(timeZoneId)) + dateFormatter.format(value.toInt) case TimestampType => val timestampFormatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(timeZoneId)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index b9a6b998e52fe..d3cd46e4e6b89 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -208,20 +208,19 @@ SELECT date '5874898-01-01'; -- out of range SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; --- [SPARK-28141] Date type can not accept special values --- SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL; --- SELECT date 'yesterday' - date 'today' AS "One day"; +SELECT date 'yesterday' - date 'today' AS `One day`; --- SELECT date 'today' - date 'tomorrow' AS "One day"; +SELECT date 'today' - date 'tomorrow' AS `One day`; --- SELECT date 'yesterday' - date 'tomorrow' AS "Two days"; +SELECT date 'yesterday' - date 'tomorrow' AS `Two days`; --- SELECT date 'tomorrow' - date 'today' AS "One day"; +SELECT date 'tomorrow' - date 'today' AS `One day`; --- SELECT date 'today' - date 'yesterday' AS "One day"; +SELECT date 'today' - date 'yesterday' AS `One day`; --- SELECT date 'tomorrow' - date 'yesterday' AS "Two days"; +SELECT date 'tomorrow' - date 'yesterday' AS `Two days`; -- [SPARK-28017] Enhance date EXTRACT -- @@ -290,7 +289,7 @@ SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-01-01 BC --- [SPARK-28141] Date type can not accept special values +-- [SPARK-29006] Support special date/timestamp values `infinity`/`-infinity` -- -- test infinity -- diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index 083832007d618..29fcf61bd5b78 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 91 +-- Number of queries: 98 -- !query 0 @@ -502,352 +502,422 @@ struct -- !query 47 -SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL -- !query 47 schema -struct +struct -- !query 47 output -0 +-4585 +-4650 +11048 +11049 +11050 +24934 +25300 +25667 +9554 +9555 +9556 +9557 +9920 +9921 +9922 -- !query 48 -SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') +SELECT date 'yesterday' - date 'today' AS `One day` -- !query 48 schema -struct +struct -- !query 48 output -0 +-1 -- !query 49 -SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) +SELECT date 'today' - date 'tomorrow' AS `One day` -- !query 49 schema -struct +struct -- !query 49 output --2 +-1 -- !query 50 -SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) +SELECT date 'yesterday' - date 'tomorrow' AS `Two days` -- !query 50 schema -struct +struct -- !query 50 output --1 +-2 -- !query 51 -SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) +SELECT date 'tomorrow' - date 'today' AS `One day` -- !query 51 schema -struct +struct -- !query 51 output --1 +1 -- !query 52 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') +SELECT date 'today' - date 'yesterday' AS `One day` -- !query 52 schema -struct +struct -- !query 52 output 1 -- !query 53 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') +SELECT date 'tomorrow' - date 'yesterday' AS `Two days` -- !query 53 schema -struct +struct -- !query 53 output -1 +2 -- !query 54 -SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') +SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 54 schema -struct +struct -- !query 54 output -19 +0 -- !query 55 -SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') +SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema -struct +struct -- !query 55 output -20 +0 -- !query 56 -SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') +SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) -- !query 56 schema -struct +struct -- !query 56 output -20 +-2 -- !query 57 -SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') +SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) -- !query 57 schema -struct +struct -- !query 57 output -21 +-1 -- !query 58 -SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True +SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 58 schema -struct +struct -- !query 58 output -true +-1 -- !query 59 -SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') -- !query 59 schema -struct +struct -- !query 59 output --1 +1 -- !query 60 -SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') -- !query 60 schema -struct +struct -- !query 60 output 1 -- !query 61 -SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') +SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') -- !query 61 schema -struct +struct -- !query 61 output -1 +19 -- !query 62 -SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') +SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') -- !query 62 schema -struct +struct -- !query 62 output -2 +20 -- !query 63 -SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') +SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') -- !query 63 schema -struct +struct -- !query 63 output -2 +20 -- !query 64 -SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') +SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') -- !query 64 schema -struct +struct -- !query 64 output -3 +21 -- !query 65 -SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) +SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True -- !query 65 schema -struct +struct -- !query 65 output -3 +true -- !query 66 -SELECT EXTRACT(DECADE FROM DATE '1994-12-25') +SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 66 schema -struct +struct -- !query 66 output -199 +-1 -- !query 67 -SELECT EXTRACT(DECADE FROM DATE '0010-01-01') +SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') -- !query 67 schema -struct +struct -- !query 67 output 1 -- !query 68 -SELECT EXTRACT(DECADE FROM DATE '0009-12-31') +SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') -- !query 68 schema -struct +struct -- !query 68 output -0 +1 -- !query 69 -SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') -- !query 69 schema -struct +struct -- !query 69 output -0 +2 -- !query 70 -SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') -- !query 70 schema -struct +struct -- !query 70 output --1 +2 -- !query 71 -SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') -- !query 71 schema -struct +struct -- !query 71 output --1 +3 -- !query 72 -SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) -- !query 72 schema -struct +struct -- !query 72 output --2 +3 -- !query 73 -SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True +SELECT EXTRACT(DECADE FROM DATE '1994-12-25') -- !query 73 schema -struct +struct -- !query 73 output -true +199 -- !query 74 -SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM DATE '0010-01-01') -- !query 74 schema -struct +struct -- !query 74 output -20 +1 -- !query 75 -SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM DATE '0009-12-31') -- !query 75 schema -struct +struct -- !query 75 output -1001-01-01 00:07:02 +0 -- !query 76 -SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') +SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) -- !query 76 schema -struct +struct -- !query 76 output -1001-01-01 00:07:02 +0 -- !query 77 -SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query 77 schema -struct +struct -- !query 77 output -1901-01-01 00:00:00 +-1 -- !query 78 -SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') +SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) -- !query 78 schema -struct +struct -- !query 78 output -1901-01-01 00:00:00 +-1 -- !query 79 -SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') +SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) -- !query 79 schema -struct +struct -- !query 79 output -2001-01-01 00:00:00 +-2 -- !query 80 -SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') +SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True -- !query 80 schema -struct +struct -- !query 80 output -0001-01-01 00:07:02 +true -- !query 81 -SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 81 schema -struct +struct -- !query 81 output --0099-01-01 00:07:02 +20 -- !query 82 -SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 82 schema -struct +struct -- !query 82 output -1990-01-01 00:00:00 +1001-01-01 00:07:02 -- !query 83 -SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') -- !query 83 schema -struct +struct -- !query 83 output -0000-01-01 00:07:02 +1001-01-01 00:07:02 -- !query 84 -SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 84 schema -struct +struct -- !query 84 output --0010-01-01 00:07:02 +1901-01-01 00:00:00 -- !query 85 -select make_date(2013, 7, 15) +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') -- !query 85 schema -struct +struct -- !query 85 output -2013-07-15 +1901-01-01 00:00:00 -- !query 86 -select make_date(-44, 3, 15) +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') -- !query 86 schema -struct +struct -- !query 86 output --0044-03-15 +2001-01-01 00:00:00 -- !query 87 -select make_date(2013, 2, 30) +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') -- !query 87 schema -struct +struct -- !query 87 output -NULL +0001-01-01 00:07:02 -- !query 88 -select make_date(2013, 13, 1) +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) -- !query 88 schema -struct +struct -- !query 88 output -NULL +-0099-01-01 00:07:02 -- !query 89 -select make_date(2013, 11, -1) +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') -- !query 89 schema -struct +struct -- !query 89 output -NULL +1990-01-01 00:00:00 -- !query 90 -DROP TABLE DATE_TBL +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') -- !query 90 schema -struct<> +struct -- !query 90 output +0000-01-01 00:07:02 + + +-- !query 91 +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +-- !query 91 schema +struct +-- !query 91 output +-0010-01-01 00:07:02 + + +-- !query 92 +select make_date(2013, 7, 15) +-- !query 92 schema +struct +-- !query 92 output +2013-07-15 + + +-- !query 93 +select make_date(-44, 3, 15) +-- !query 93 schema +struct +-- !query 93 output +-0044-03-15 + + +-- !query 94 +select make_date(2013, 2, 30) +-- !query 94 schema +struct +-- !query 94 output +NULL + + +-- !query 95 +select make_date(2013, 13, 1) +-- !query 95 schema +struct +-- !query 95 output +NULL + + +-- !query 96 +select make_date(2013, 11, -1) +-- !query 96 schema +struct +-- !query 96 output +NULL + + +-- !query 97 +DROP TABLE DATE_TBL +-- !query 97 schema +struct<> +-- !query 97 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 1094d7d23e5ea..d34e50518348a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale @@ -191,4 +191,13 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) } } + + test("special date values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(specialValue).toDS() + val readback = input.select(from_csv($"value", lit("d date"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Date](0).getTime >= 0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index c61c8109ee8e6..e55d2bbe00e63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale @@ -618,4 +618,13 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) } } + + test("special date values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(s"""{"d": "$specialValue"}""").toDS() + val readback = input.select(from_json($"value", lit("d date"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Date](0).getTime >= 0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 0a85e3cdeaf1d..138336dc7e333 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -58,7 +58,7 @@ abstract class ParquetPartitionDiscoverySuite val defaultPartitionName = ExternalCatalogUtils.DEFAULT_PARTITION_NAME val timeZoneId = ZoneId.systemDefault() - val df = DateFormatter() + val df = DateFormatter(timeZoneId) val tf = TimestampFormatter(timestampPartitionPattern, timeZoneId) protected override def beforeAll(): Unit = { From a838dbd2f97389fda096093cbe6832a07fab4a0d Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sun, 22 Sep 2019 21:39:30 +0900 Subject: [PATCH 83/87] [SPARK-27463][PYTHON][FOLLOW-UP] Run the tests of Cogrouped pandas UDF ### What changes were proposed in this pull request? This is a followup for https://github.com/apache/spark/pull/24981 Seems we mistakenly didn't added `test_pandas_udf_cogrouped_map` into `modules.py`. So we don't have official test results against that PR. ``` ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_agg ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_map ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_scalar ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_window Finished test(python3.6): pyspark.sql.tests.test_pandas_udf (21s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_map (49s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_window (58s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_scalar (82s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_agg (105s) ... ``` If tests fail, we should revert that PR. ### Why are the changes needed? Relevant tests should be ran. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Jenkins tests. Closes #25890 from HyukjinKwon/SPARK-28840. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/sparktestsupport/modules.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index d4e6b6749010a..c7ea065b28ed8 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -372,6 +372,7 @@ def __hash__(self): "pyspark.sql.tests.test_functions", "pyspark.sql.tests.test_group", "pyspark.sql.tests.test_pandas_udf", + "pyspark.sql.tests.test_pandas_udf_cogrouped_map", "pyspark.sql.tests.test_pandas_udf_grouped_agg", "pyspark.sql.tests.test_pandas_udf_grouped_map", "pyspark.sql.tests.test_pandas_udf_scalar", From fe4bee8fd825f8b123569ded6e50e567f5cf9e2c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 22 Sep 2019 11:17:47 -0700 Subject: [PATCH 84/87] [SPARK-29162][SQL] Simplify NOT(IsNull(x)) and NOT(IsNotNull(x)) ### What changes were proposed in this pull request? Rewrite ``` NOT isnull(x) -> isnotnull(x) NOT isnotnull(x) -> isnull(x) ``` ### Why are the changes needed? Make LogicalPlan more readable and useful for query canonicalization. Make same condition equal when judge query canonicalization equal ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Newly added UTs. Closes #25878 from AngersZhuuuu/SPARK-29162. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/optimizer/expressions.scala | 3 +++ .../sql/catalyst/optimizer/BooleanSimplificationSuite.scala | 5 +++++ 2 files changed, 8 insertions(+) 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 39709529c00d3..0a6737ba42118 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 @@ -373,6 +373,9 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case Not(a And b) => Or(Not(a), Not(b)) case Not(Not(e)) => e + + case Not(IsNull(e)) => IsNotNull(e) + case Not(IsNotNull(e)) => IsNull(e) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index a0de5f6930958..a8b8417754b00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -239,6 +239,11 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) } + test("simplify NOT(IsNull(x)) and NOT(IsNotNull(x))") { + checkCondition(Not(IsNotNull('b)), IsNull('b)) + checkCondition(Not(IsNull('b)), IsNotNull('b)) + } + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation()).analyze val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation()).analyze) From 76bc9db7493a53fc89d9549ac366226701491bc5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 22 Sep 2019 13:53:21 -0700 Subject: [PATCH 85/87] [SPARK-29191][TESTS][SQL] Add tag ExtendedSQLTest for SQLQueryTestSuite ### What changes were proposed in this pull request? This PR aims to add tag `ExtendedSQLTest` for `SQLQueryTestSuite`. This doesn't affect our Jenkins test coverage. Instead, this tag gives us an ability to parallelize them by splitting this test suite and the other suites. ### Why are the changes needed? `SQLQueryTestSuite` takes 45 mins alone because it has many SQL scripts to run. time ### Does this PR introduce any user-facing change? No. ### How was this patch tested? ``` build/sbt "sql/test-only *.SQLQueryTestSuite" -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest ... [info] SQLQueryTestSuite: [info] ScalaTest [info] Run completed in 3 seconds, 147 milliseconds. [info] Total number of tests run: 0 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0 [info] No tests were executed. [info] Passed: Total 0, Failed 0, Errors 0, Passed 0 [success] Total time: 22 s, completed Sep 20, 2019 12:23:13 PM ``` Closes #25872 from dongjoon-hyun/SPARK-29191. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/tags/ExtendedSQLTest.java | 30 +++++++++++++++++++ .../apache/spark/sql/SQLQueryTestSuite.scala | 2 ++ 2 files changed, 32 insertions(+) create mode 100644 common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java diff --git a/common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java b/common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java new file mode 100644 index 0000000000000..1c0fff1b4045d --- /dev/null +++ b/common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.tags; + +import org.scalatest.TagAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface ExtendedSQLTest { } 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 f32575794f946..1a41dd95a5700 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 @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCo import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType +import org.apache.spark.tags.ExtendedSQLTest /** * End-to-end test cases for SQL queries. @@ -104,6 +105,7 @@ import org.apache.spark.sql.types.StructType * Therefore, UDF test cases should have single input and output files but executed by three * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. */ +@ExtendedSQLTest class SQLQueryTestSuite extends QueryTest with SharedSparkSession { import IntegratedUDFTestUtils._ From 51d350942865302e33480c83324704a54a24a494 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 22 Sep 2019 14:12:06 -0700 Subject: [PATCH 86/87] [SPARK-28599][SQL] Fix `Execution Time` and `Duration` column sorting for ThriftServerSessionPage ### What changes were proposed in this pull request? This PR add support sorting `Execution Time` and `Duration` columns for `ThriftServerSessionPage`. ### Why are the changes needed? Previously, it's not sorted correctly. ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? Manually do the following and test sorting on those columns in the Spark Thrift Server Session Page. ``` $ sbin/start-thriftserver.sh $ bin/beeline -u jdbc:hive2://localhost:10000 0: jdbc:hive2://localhost:10000> create table t(a int); +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (0.521 seconds) 0: jdbc:hive2://localhost:10000> select * from t; +----+--+ | a | +----+--+ +----+--+ No rows selected (0.772 seconds) 0: jdbc:hive2://localhost:10000> show databases; +---------------+--+ | databaseName | +---------------+--+ | default | +---------------+--+ 1 row selected (0.249 seconds) ``` **Sorted by `Execution Time` column**: ![image](https://user-images.githubusercontent.com/5399861/65387476-53038900-dd7a-11e9-885c-fca80287f550.png) **Sorted by `Duration` column**: ![image](https://user-images.githubusercontent.com/5399861/65387481-6e6e9400-dd7a-11e9-9318-f917247efaa8.png) Closes #25892 from wangyum/SPARK-28599. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../hive/thriftserver/ui/ThriftServerSessionPage.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 912b658eb36b9..b752634aef956 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 @@ -100,8 +100,12 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) {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)))} + + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + + + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} + {info.statement} {info.state} {errorMessageCell(detail)} From 7a2ea58e782cde4c9b7ec887c6eeb642659d8614 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 22 Sep 2019 14:47:42 -0700 Subject: [PATCH 87/87] [SPARK-29084][SQL][TESTS] Check method bytecode size in BenchmarkQueryTest ### What changes were proposed in this pull request? This pr proposes to check method bytecode size in `BenchmarkQueryTest`. This metric is critical for performance numbers. ### Why are the changes needed? For performance checks ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25788 from maropu/CheckMethodSize. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/BenchmarkQueryTest.scala | 11 ++++++++--- .../scala/org/apache/spark/sql/TPCDSQuerySuite.scala | 10 ++++++++-- .../execution/LogicalPlanTagInSparkPlanSuite.scala | 5 +++-- 3 files changed, 19 insertions(+), 7 deletions(-) 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 3fcb9892800b6..9bb741bee4e40 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.internal.config.Tests.IS_TESTING -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator} +import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeFormatter, CodeGenerator} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.test.SharedSparkSession @@ -48,7 +48,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { RuleExecutor.resetMetrics() } - protected def checkGeneratedCode(plan: SparkPlan): Unit = { + protected def checkGeneratedCode(plan: SparkPlan, checkMethodCodeSize: Boolean = true): Unit = { val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() plan foreach { case s: WholeStageCodegenExec => @@ -57,7 +57,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { } codegenSubtrees.toSeq.foreach { subtree => val code = subtree.doCodeGen()._2 - try { + val (_, ByteCodeStats(maxMethodCodeSize, _, _)) = try { // Just check the generated code can be properly compiled CodeGenerator.compile(code) } catch { @@ -72,6 +72,11 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { """.stripMargin throw new Exception(msg, e) } + + assert(!checkMethodCodeSize || + maxMethodCodeSize <= CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT, + s"too long generated codes found in the WholeStageCodegenExec subtree (id=${subtree.id}) " + + s"and JIT optimization might not work:\n${subtree.treeString}") } } } 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 a668434a68aff..11e66e878c149 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 @@ -82,13 +82,19 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSSchema { "q3", "q7", "q10", "q19", "q27", "q34", "q42", "q43", "q46", "q52", "q53", "q55", "q59", "q63", "q65", "q68", "q73", "q79", "q89", "q98", "ss_max") + // List up the known queries having too large code in a generated function. + // A JIRA file for `modified-q3` is as follows; + // [SPARK-29128] Split predicate code in OR expressions + val blackListForMethodCodeSizeCheck = Set("modified-q3") + modifiedTPCDSQueries.foreach { name => val queryString = resourceToString(s"tpcds-modifiedQueries/$name.sql", classLoader = Thread.currentThread().getContextClassLoader) - test(s"modified-$name") { + val testName = s"modified-$name" + test(testName) { // check the plans can be properly generated val plan = sql(queryString).queryExecution.executedPlan - checkGeneratedCode(plan) + checkGeneratedCode(plan, !blackListForMethodCodeSizeCheck.contains(testName)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala index aa83b9b11dcfc..44af422b90837 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala @@ -32,8 +32,9 @@ import org.apache.spark.sql.execution.window.WindowExec class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { - override protected def checkGeneratedCode(plan: SparkPlan): Unit = { - super.checkGeneratedCode(plan) + override protected def checkGeneratedCode( + plan: SparkPlan, checkMethodCodeSize: Boolean = true): Unit = { + super.checkGeneratedCode(plan, checkMethodCodeSize) checkLogicalPlanTag(plan) }