spark-instrumented-optimizer/docs/sql-migration-guide.md

1049 lines
69 KiB
Markdown
Raw Normal View History

---
layout: global
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
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
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
---
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
* Table of contents
{:toc}
## Upgrading from Spark SQL 2.4 to 3.0
[SPARK-28885][SQL] Follow ANSI store assignment rules in table insertion by default ### What changes were proposed in this pull request? When inserting a value into a column with the different data type, Spark performs type coercion. Currently, we support 3 policies for the store assignment rules: ANSI, legacy and strict, which can be set via the option "spark.sql.storeAssignmentPolicy": 1. ANSI: Spark performs the type coercion as per ANSI SQL. In practice, the behavior is mostly the same as PostgreSQL. It disallows certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean`. It will throw a runtime exception if the value is out-of-range(overflow). 2. Legacy: Spark allows the type coercion as long as it is a valid `Cast`, which is very loose. E.g., converting either `string` to `int` or `double` to `boolean` is allowed. It is the current behavior in Spark 2.x for compatibility with Hive. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of Byte type, the result is 1. 3. Strict: Spark doesn't allow any possible precision loss or data truncation in store assignment, e.g., converting either `double` to `int` or `decimal` to `double` is allowed. The rules are originally for Dataset encoder. As far as I know, no mainstream DBMS is using this policy by default. Currently, the V1 data source uses "Legacy" policy by default, while V2 uses "Strict". This proposal is to use "ANSI" policy by default for both V1 and V2 in Spark 3.0. ### Why are the changes needed? Following the ANSI SQL standard is most reasonable among the 3 policies. ### Does this PR introduce any user-facing change? Yes. The default store assignment policy is ANSI for both V1 and V2 data sources. ### How was this patch tested? Unit test Closes #26107 from gengliangwang/ansiPolicyAsDefault. Authored-by: Gengliang Wang <gengliang.wang@databricks.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-15 13:41:37 -04:00
- Since Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception will be thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and earlier, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior.
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
- 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 <table>` or `FROM <table> UNION ALL FROM <table>` are supported by accident. In hive-style `FROM <table> SELECT <expr>`, 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.
- Creating of typed `TIMESTAMP` and `DATE` literals from strings. Since Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. In Spark version 2.4 and earlier, the `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()` functions are used for the same purpose.
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
- 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.enabled` to `true`.
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
- 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, 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.enabled` to `false`.
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
- 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:
<table class="table">
<tr>
<th>
<b>Operation</b>
</th>
<th>
<b>Result prior to Spark 3.0</b>
</th>
<th>
<b>Result starting Spark 3.0</b>
</th>
</tr>
<tr>
<td>
CAST('infinity' AS DOUBLE)<br>
CAST('+infinity' AS DOUBLE)<br>
CAST('inf' AS DOUBLE)<br>
CAST('+inf' AS DOUBLE)<br>
</td>
<td>
NULL
</td>
<td>
Double.PositiveInfinity
</td>
</tr>
<tr>
<td>
CAST('-infinity' AS DOUBLE)<br>
CAST('-inf' AS DOUBLE)<br>
</td>
<td>
NULL
</td>
<td>
Double.NegativeInfinity
</td>
</tr>
<tr>
<td>
CAST('infinity' AS FLOAT)<br>
CAST('+infinity' AS FLOAT)<br>
CAST('inf' AS FLOAT)<br>
CAST('+inf' AS FLOAT)<br>
</td>
<td>
NULL
</td>
<td>
Float.PositiveInfinity
</td>
</tr>
<tr>
<td>
CAST('-infinity' AS FLOAT)<br>
CAST('-inf' AS FLOAT)<br>
</td>
<td>
NULL
</td>
<td>
Float.NegativeInfinity
</td>
</tr>
<tr>
<td>
CAST('nan' AS DOUBLE)
</td>
<td>
NULL
</td>
<td>
Double.NaN
</td>
</tr>
<tr>
<td>
CAST('nan' AS FLOAT)
</td>
<td>
NULL
</td>
<td>
Float.NaN
</td>
</tr>
</table>
- Since Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that will be converted to ordinary date or timestamp values when read. The following string values are supported for dates:
- `epoch [zoneId]` - 1970-01-01
- `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone`
- `yesterday [zoneId]` - the current date - 1
- `tomorrow [zoneId]` - the current date + 1
- `now` - the date of running the current query. It has the same notion as today
For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values:
- `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero)
- `today [zoneId]` - midnight today
- `yesterday [zoneId]` - midnight yesterday
- `tomorrow [zoneId]` - midnight tomorrow
- `now` - current query start time
For example `SELECT timestamp 'tomorrow';`.
- Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`.
- Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception.
- Since Spark 3.0, when casting interval values to string type, there is no "interval" prefix, e.g. `1 days 2 hours`. In Spark version 2.4 and earlier, the string contains the "interval" prefix like `interval 1 days 2 hours`.
- Since Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces (<= ASCII 32) will be trimmed before converted to these type values, e.g. `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, while casting string to integrals and booleans, it will not trim the whitespaces from both ends, the foregoing results will be `null`, while to datetimes, only the trailing spaces (= ASCII 32) will be removed.
[SPARK-28023][SQL] Add trim logic in UTF8String's toInt/toLong to make it consistent with other string-numeric casting ### What changes were proposed in this pull request? Modify `UTF8String.toInt/toLong` to support trim spaces for both sides before converting it to byte/short/int/long. With this kind of "cheap" trim can help improve performance for casting string to integrals. The idea is from https://github.com/apache/spark/pull/24872#issuecomment-556917834 ### Why are the changes needed? make the behavior consistent. ### Does this PR introduce any user-facing change? yes, cast string to an integral type, and binary comparison between string and integrals will trim spaces first. their behavior will be consistent with float and double. ### How was this patch tested? 1. add ut. 2. benchmark tests the benchmark is modified based on https://github.com/apache/spark/pull/24872#issuecomment-503827016 ```scala /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark /** * Benchmark trim the string when casting string type to Boolean/Numeric types. * To run this benchmark: * {{{ * 1. without sbt: * bin/spark-submit --class <this class> --jars <spark core test jar> <spark sql test jar> * 2. build/sbt "sql/test:runMain <this class>" * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>" * Results will be written to "benchmarks/CastBenchmark-results.txt". * }}} */ object CastBenchmark extends SqlBasedBenchmark { This conversation was marked as resolved by yaooqinn override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val title = "Cast String to Integral" runBenchmark(title) { withTempPath { dir => val N = 500L << 14 val df = spark.range(N) val types = Seq("int", "long") (1 to 5).by(2).foreach { i => df.selectExpr(s"concat(id, '${" " * i}') as str") .write.mode("overwrite").parquet(dir + i.toString) } val benchmark = new Benchmark(title, N, minNumIters = 5, output = output) Seq(true, false).foreach { trim => types.foreach { t => val str = if (trim) "trim(str)" else "str" val expr = s"cast($str as $t) as c_$t" (1 to 5).by(2).foreach { i => benchmark.addCase(expr + s" - with $i spaces") { _ => spark.read.parquet(dir + i.toString).selectExpr(expr).collect() } } } } benchmark.run() } } } } ``` #### benchmark result. normal trim v.s. trim in toInt/toLong ```java ================================================================================================ Cast String to Integral ================================================================================================ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 Intel(R) Core(TM) i5-5287U CPU 2.90GHz Cast String to Integral: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast(trim(str) as int) as c_int - with 1 spaces 10220 12994 1337 0.8 1247.5 1.0X cast(trim(str) as int) as c_int - with 3 spaces 4763 8356 357 1.7 581.4 2.1X cast(trim(str) as int) as c_int - with 5 spaces 4791 8042 NaN 1.7 584.9 2.1X cast(trim(str) as long) as c_long - with 1 spaces 4014 6755 NaN 2.0 490.0 2.5X cast(trim(str) as long) as c_long - with 3 spaces 4737 6938 NaN 1.7 578.2 2.2X cast(trim(str) as long) as c_long - with 5 spaces 4478 6919 1404 1.8 546.6 2.3X cast(str as int) as c_int - with 1 spaces 4443 6222 NaN 1.8 542.3 2.3X cast(str as int) as c_int - with 3 spaces 3659 3842 170 2.2 446.7 2.8X cast(str as int) as c_int - with 5 spaces 4372 7996 NaN 1.9 533.7 2.3X cast(str as long) as c_long - with 1 spaces 3866 5838 NaN 2.1 471.9 2.6X cast(str as long) as c_long - with 3 spaces 3793 5449 NaN 2.2 463.0 2.7X cast(str as long) as c_long - with 5 spaces 4947 5961 1198 1.7 603.9 2.1X ``` Closes #26622 from yaooqinn/cheapstringtrim. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-22 06:32:27 -05:00
[SPARK-29956][SQL] A literal number with an exponent should be parsed to Double ### What changes were proposed in this pull request? For a literal number with an exponent(e.g. 1e-45, 1E2), we'd parse it to Double by default rather than Decimal. And user could still use `spark.sql.legacy.exponentLiteralToDecimal.enabled=true` to fall back to previous behavior. ### Why are the changes needed? According to ANSI standard of SQL, we see that the (part of) definition of `literal` : ``` <approximate numeric literal> ::= <mantissa> E <exponent> ``` which indicates that a literal number with an exponent should be approximate numeric(e.g. Double) rather than exact numeric(e.g. Decimal). And when we test Presto, we found that Presto also conforms to this standard: ``` presto:default> select typeof(1E2); _col0 -------- double (1 row) ``` ``` presto:default> select typeof(1.2); _col0 -------------- decimal(2,1) (1 row) ``` We also find that, actually, literals like `1E2` are parsed as Double before Spark2.1, but changed to Decimal after #14828 due to *The difference between the two confuses most users* as it said. But we also see support(from DB2 test) of original behavior at #14828 (comment). Although, we also see that PostgreSQL has its own implementation: ``` postgres=# select pg_typeof(1E2); pg_typeof ----------- numeric (1 row) postgres=# select pg_typeof(1.2); pg_typeof ----------- numeric (1 row) ``` We still think that Spark should also conform to this standard while considering SQL standard and Spark own history and majority DBMS and also user experience. ### Does this PR introduce any user-facing change? Yes. For `1E2`, before this PR: ``` scala> spark.sql("select 1E2") res0: org.apache.spark.sql.DataFrame = [1E+2: decimal(1,-2)] ``` After this PR: ``` scala> spark.sql("select 1E2") res0: org.apache.spark.sql.DataFrame = [100.0: double] ``` And for `1E-45`, before this PR: ``` org.apache.spark.sql.catalyst.parser.ParseException: decimal can only support precision up to 38 == SQL == select 1E-45 at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:131) at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76) at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605) ... 47 elided ``` after this PR: ``` scala> spark.sql("select 1E-45"); res1: org.apache.spark.sql.DataFrame = [1.0E-45: double] ``` And before this PR, user may feel super weird to see that `select 1e40` works but `select 1e-40 fails`. And now, both of them work well. ### How was this patch tested? updated `literals.sql.out` and `ansi/literals.sql.out` Closes #26595 from Ngone51/SPARK-29956. Authored-by: wuyi <ngone_5451@163.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-01 22:34:56 -05:00
- Since Spark 3.0, numbers written in scientific notation(e.g. `1E2`) would be parsed as Double. In Spark version 2.4 and earlier, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`.
[SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column ## What changes were proposed in this pull request? [HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063. > HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1). However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0. The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however. **Spark SQL**: ```sql // bin/spark-sql spark-sql> select cast(1 as decimal(38, 18)); 1 spark-sql> // bin/beeline 0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18)); +----------------------------+--+ | CAST(1 AS DECIMAL(38,18)) | +----------------------------+--+ | 1.000000000000000000 | +----------------------------+--+ // bin/spark-shell scala> spark.sql("select cast(1 as decimal(38, 18))").show(false) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ |1.000000000000000000 | +-------------------------+ // bin/pyspark >>> spark.sql("select cast(1 as decimal(38, 18))").show() +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ // bin/sparkR > showDF(sql("SELECT cast(1 as decimal(38, 18))")) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ ``` **PostgreSQL**: ```sql postgres=# select cast(1 as decimal(38, 18)); numeric ---------------------- 1.000000000000000000 (1 row) ``` **Presto**: ```sql presto> select cast(1 as decimal(38, 18)); _col0 ---------------------- 1.000000000000000000 (1 row) ``` ## How was this patch tested? unit tests and manual test: ```sql spark-sql> select cast(1 as decimal(38, 18)); 1.000000000000000000 ``` Spark SQL Upgrading Guide: ![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png) Closes #26697 from wangyum/SPARK-28461. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-01 19:02:39 -05:00
- Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example:
<table class="table">
<tr>
<th>
<b>Query</b>
</th>
<th>
<b>Spark 2.4 or Prior</b>
</th>
<th>
<b>Spark 3.0</b>
</th>
</tr>
<tr>
<td>
<code>SELECT CAST(1 AS decimal(38, 18));</code>
</td>
<td>
<code>1</code>
</td>
<td>
<code>1.000000000000000000</code>
</td>
</tr>
</table>
[SPARK-30098][SQL] Use default datasource as provider for CREATE TABLE syntax ### What changes were proposed in this pull request? In this PR, we propose to use the value of `spark.sql.source.default` as the provider for `CREATE TABLE` syntax instead of `hive` in Spark 3.0. And to help the migration, we introduce a legacy conf `spark.sql.legacy.respectHiveDefaultProvider.enabled` and set its default to `false`. ### Why are the changes needed? 1. Currently, `CREATE TABLE` syntax use hive provider to create table while `DataFrameWriter.saveAsTable` API using the value of `spark.sql.source.default` as a provider to create table. It would be better to make them consistent. 2. User may gets confused in some cases. For example: ``` CREATE TABLE t1 (c1 INT) USING PARQUET; CREATE TABLE t2 (c1 INT); ``` In these two DDLs, use may think that `t2` should also use parquet as default provider since Spark always advertise parquet as the default format. However, it's hive in this case. On the other hand, if we omit the USING clause in a CTAS statement, we do pick parquet by default if `spark.sql.hive.convertCATS=true`: ``` CREATE TABLE t3 USING PARQUET AS SELECT 1 AS VALUE; CREATE TABLE t4 AS SELECT 1 AS VALUE; ``` And these two cases together can be really confusing. 3. Now, Spark SQL is very independent and popular. We do not need to be fully consistent with Hive's behavior. ### Does this PR introduce any user-facing change? Yes, before this PR, using `CREATE TABLE` syntax will use hive provider. But now, it use the value of `spark.sql.source.default` as its provider. ### How was this patch tested? Added tests in `DDLParserSuite` and `HiveDDlSuite`. Closes #26736 from Ngone51/dev-create-table-using-parquet-by-default. Lead-authored-by: wuyi <yi.wu@databricks.com> Co-authored-by: yi.wu <yi.wu@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-06 13:15:25 -05:00
- Since Spark 3.0, CREATE TABLE without a specific provider will use the value of `spark.sql.sources.default` as its provider. In Spark version 2.4 and earlier, it was hive. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createHiveTableByDefault.enabled` to `true`.
[SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column ## What changes were proposed in this pull request? [HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063. > HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1). However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0. The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however. **Spark SQL**: ```sql // bin/spark-sql spark-sql> select cast(1 as decimal(38, 18)); 1 spark-sql> // bin/beeline 0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18)); +----------------------------+--+ | CAST(1 AS DECIMAL(38,18)) | +----------------------------+--+ | 1.000000000000000000 | +----------------------------+--+ // bin/spark-shell scala> spark.sql("select cast(1 as decimal(38, 18))").show(false) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ |1.000000000000000000 | +-------------------------+ // bin/pyspark >>> spark.sql("select cast(1 as decimal(38, 18))").show() +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ // bin/sparkR > showDF(sql("SELECT cast(1 as decimal(38, 18))")) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ ``` **PostgreSQL**: ```sql postgres=# select cast(1 as decimal(38, 18)); numeric ---------------------- 1.000000000000000000 (1 row) ``` **Presto**: ```sql presto> select cast(1 as decimal(38, 18)); _col0 ---------------------- 1.000000000000000000 (1 row) ``` ## How was this patch tested? unit tests and manual test: ```sql spark-sql> select cast(1 as decimal(38, 18)); 1.000000000000000000 ``` Spark SQL Upgrading Guide: ![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png) Closes #26697 from wangyum/SPARK-28461. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-01 19:02:39 -05:00
[SPARK-30083][SQL] visitArithmeticUnary should wrap PLUS case with UnaryPositive for type checking ### What changes were proposed in this pull request? `UnaryPositive` only accepts numeric and interval as we defined, but what we do for this in `AstBuider.visitArithmeticUnary` is just bypassing it. This should not be omitted for the type checking requirement. ### Why are the changes needed? bug fix, you can find a pre-discussion here https://github.com/apache/spark/pull/26578#discussion_r347350398 ### Does this PR introduce any user-facing change? yes, +non-numeric-or-interval is now invalid. ``` -- !query 14 select +date '1900-01-01' -- !query 14 schema struct<DATE '1900-01-01':date> -- !query 14 output 1900-01-01 -- !query 15 select +timestamp '1900-01-01' -- !query 15 schema struct<TIMESTAMP '1900-01-01 00:00:00':timestamp> -- !query 15 output 1900-01-01 00:00:00 -- !query 16 select +map(1, 2) -- !query 16 schema struct<map(1, 2):map<int,int>> -- !query 16 output {1:2} -- !query 17 select +array(1,2) -- !query 17 schema struct<array(1, 2):array<int>> -- !query 17 output [1,2] -- !query 18 select -'1' -- !query 18 schema struct<(- CAST(1 AS DOUBLE)):double> -- !query 18 output -1.0 -- !query 19 select -X'1' -- !query 19 schema struct<> -- !query 19 output org.apache.spark.sql.AnalysisException cannot resolve '(- X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 -- !query 20 select +X'1' -- !query 20 schema struct<X'01':binary> -- !query 20 output ``` ### How was this patch tested? add ut check Closes #26716 from yaooqinn/SPARK-30083. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-03 10:42:21 -05:00
- Since Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation will be coerced to double value, e.g. `+'1'` results `1.0`. In Spark version 2.4 and earlier, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, e.g. `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, e.g. in Spark 2.4, the result of `+'1'` is string `1`.
[SPARK-29864][SPARK-29920][SQL] Strict parsing of day-time strings to intervals ### What changes were proposed in this pull request? In the PR, I propose new implementation of `fromDayTimeString` which strictly parses strings in day-time formats to intervals. New implementation accepts only strings that match to a pattern defined by the `from` and `to`. Here is the mapping of user's bounds and patterns: - `[+|-]D+ H[H]:m[m]:s[s][.SSSSSSSSS]` for **DAY TO SECOND** - `[+|-]D+ H[H]:m[m]` for **DAY TO MINUTE** - `[+|-]D+ H[H]` for **DAY TO HOUR** - `[+|-]H[H]:m[m]s[s][.SSSSSSSSS]` for **HOUR TO SECOND** - `[+|-]H[H]:m[m]` for **HOUR TO MINUTE** - `[+|-]m[m]:s[s][.SSSSSSSSS]` for **MINUTE TO SECOND** Closes #26327 Closes #26358 ### Why are the changes needed? - Improve user experience with Spark SQL, and respect to the bound specified by users. - Behave the same as other broadly used DBMS - Oracle and MySQL. ### Does this PR introduce any user-facing change? Yes, before: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; interval 1 weeks 3 days 11 hours 12 minutes ``` After: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; Error in query: requirement failed: Interval string must match day-time format of '^(?<sign>[+|-])?(?<hour>\d{1,2}):(?<minute>\d{1,2})$': 10 11:12:13.123(line 1, pos 16) == SQL == SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE ----------------^^^ ``` ### How was this patch tested? - Added tests to `IntervalUtilsSuite` - By `ExpressionParserSuite` - Updated `literals.sql` Closes #26473 from MaxGekk/strict-from-daytime-string. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-11 12:08:53 -05:00
- Since Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`.
[SPARK-30252][SQL] Disallow negative scale of Decimal ### What changes were proposed in this pull request? This PR propose to disallow negative `scale` of `Decimal` in Spark. And this PR brings two behavior changes: 1) for literals like `1.23E4BD` or `1.23E4`(with `spark.sql.legacy.exponentLiteralAsDecimal.enabled`=true, see [SPARK-29956](https://issues.apache.org/jira/browse/SPARK-29956)), we set its `(precision, scale)` to (5, 0) rather than (3, -2); 2) add negative `scale` check inside the decimal method if it exposes to set `scale` explicitly. If check fails, `AnalysisException` throws. And user could still use `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to restore the previous behavior. ### Why are the changes needed? According to SQL standard, > 4.4.2 Characteristics of numbers An exact numeric type has a precision P and a scale S. P is a positive integer that determines the number of significant digits in a particular radix R, where R is either 2 or 10. S is a non-negative integer. scale of Decimal should always be non-negative. And other mainstream databases, like Presto, PostgreSQL, also don't allow negative scale. Presto: ``` presto:default> create table t (i decimal(2, -1)); Query 20191213_081238_00017_i448h failed: line 1:30: mismatched input '-'. Expecting: <integer>, <type> create table t (i decimal(2, -1)) ``` PostgrelSQL: ``` postgres=# create table t(i decimal(2, -1)); ERROR: NUMERIC scale -1 must be between 0 and precision 2 LINE 1: create table t(i decimal(2, -1)); ^ ``` And, actually, Spark itself already doesn't allow to create table with negative decimal types using SQL: ``` scala> spark.sql("create table t(i decimal(2, -1))"); org.apache.spark.sql.catalyst.parser.ParseException: no viable alternative at input 'create table t(i decimal(2, -'(line 1, pos 28) == SQL == create table t(i decimal(2, -1)) ----------------------------^^^ at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130) at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76) at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605) ... 35 elided ``` However, it is still possible to create such table or `DatFrame` using Spark SQL programming API: ``` scala> val tb = CatalogTable( TableIdentifier("test", None), CatalogTableType.MANAGED, CatalogStorageFormat.empty, StructType(StructField("i", DecimalType(2, -1) ) :: Nil)) ``` ``` scala> spark.sql("SELECT 1.23E4BD") res2: org.apache.spark.sql.DataFrame = [1.23E+4: decimal(3,-2)] ``` while, these two different behavior could make user confused. On the other side, even if user creates such table or `DataFrame` with negative scale decimal type, it can't write data out if using format, like `parquet` or `orc`. Because these formats have their own check for negative scale and fail on it. ``` scala> spark.sql("SELECT 1.23E4BD").write.saveAsTable("parquet") 19/12/13 17:37:04 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.lang.IllegalArgumentException: Invalid DECIMAL scale: -2 at org.apache.parquet.Preconditions.checkArgument(Preconditions.java:53) at org.apache.parquet.schema.Types$BasePrimitiveBuilder.decimalMetadata(Types.java:495) at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:403) at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:309) at org.apache.parquet.schema.Types$Builder.named(Types.java:290) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:428) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:334) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.$anonfun$convert$2(ParquetSchemaConverter.scala:326) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99) at scala.collection.TraversableLike.map(TraversableLike.scala:238) at scala.collection.TraversableLike.map$(TraversableLike.scala:231) at org.apache.spark.sql.types.StructType.map(StructType.scala:99) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convert(ParquetSchemaConverter.scala:326) at org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.init(ParquetWriteSupport.scala:97) at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388) at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349) at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:150) at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:124) at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:109) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:264) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:127) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:441) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:444) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` So, I think it would be better to disallow negative scale totally and make behaviors above be consistent. ### Does this PR introduce any user-facing change? Yes, if `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false`, user couldn't create Decimal value with negative scale anymore. ### How was this patch tested? Added new tests in `ExpressionParserSuite` and `DecimalSuite`; Updated `SQLQueryTestSuite`. Closes #26881 from Ngone51/nonnegative-scale. Authored-by: yi.wu <yi.wu@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 08:09:48 -05:00
- Since Spark 3.0, negative scale of decimal is not allowed by default, e.g. data type of literal like `1E10BD` is `DecimalType(11, 0)`. In Spark version 2.4 and earlier, it was `DecimalType(2, -9)`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to `true`.
[SPARK-29864][SPARK-29920][SQL] Strict parsing of day-time strings to intervals ### What changes were proposed in this pull request? In the PR, I propose new implementation of `fromDayTimeString` which strictly parses strings in day-time formats to intervals. New implementation accepts only strings that match to a pattern defined by the `from` and `to`. Here is the mapping of user's bounds and patterns: - `[+|-]D+ H[H]:m[m]:s[s][.SSSSSSSSS]` for **DAY TO SECOND** - `[+|-]D+ H[H]:m[m]` for **DAY TO MINUTE** - `[+|-]D+ H[H]` for **DAY TO HOUR** - `[+|-]H[H]:m[m]s[s][.SSSSSSSSS]` for **HOUR TO SECOND** - `[+|-]H[H]:m[m]` for **HOUR TO MINUTE** - `[+|-]m[m]:s[s][.SSSSSSSSS]` for **MINUTE TO SECOND** Closes #26327 Closes #26358 ### Why are the changes needed? - Improve user experience with Spark SQL, and respect to the bound specified by users. - Behave the same as other broadly used DBMS - Oracle and MySQL. ### Does this PR introduce any user-facing change? Yes, before: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; interval 1 weeks 3 days 11 hours 12 minutes ``` After: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; Error in query: requirement failed: Interval string must match day-time format of '^(?<sign>[+|-])?(?<hour>\d{1,2}):(?<minute>\d{1,2})$': 10 11:12:13.123(line 1, pos 16) == SQL == SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE ----------------^^^ ``` ### How was this patch tested? - Added tests to `IntervalUtilsSuite` - By `ExpressionParserSuite` - Updated `literals.sql` Closes #26473 from MaxGekk/strict-from-daytime-string. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-11 12:08:53 -05:00
- Since Spark 3.0, the `date_add` and `date_sub` functions only accepts int, smallint, tinyint as the 2nd argument, fractional and string types are not valid anymore, e.g. `date_add(cast('1964-05-23' as date), '12.34')` will cause `AnalysisException`. In Spark version 2.4 and earlier, if the 2nd argument is fractional or string value, it will be coerced to int value, and the result will be a date value of `1964-06-04`.
[SPARK-30266][SQL] Avoid match error and int overflow in ApproximatePercentile and Percentile ### What changes were proposed in this pull request? accuracyExpression can accept Long which may cause overflow error. accuracyExpression can accept fractions which are implicitly floored. accuracyExpression can accept null which is implicitly changed to 0. percentageExpression can accept null but cause MatchError. percentageExpression can accept ArrayType(_, nullable=true) in which the nulls are implicitly changed to zeros. ##### cases ```sql select percentile_approx(10.0, 0.5, 2147483648); -- overflow and fail select percentile_approx(10.0, 0.5, 4294967297); -- overflow but success select percentile_approx(10.0, 0.5, null); -- null cast to 0 select percentile_approx(10.0, 0.5, 1.2); -- 1.2 cast to 1 select percentile_approx(10.0, null, 1); -- scala.MatchError select percentile_approx(10.0, array(0.2, 0.4, null), 1); -- null cast to zero. ``` ##### behavior before ```sql +select percentile_approx(10.0, 0.5, 2147483648) +org.apache.spark.sql.AnalysisException +cannot resolve 'percentile_approx(10.0BD, CAST(0.5BD AS DOUBLE), CAST(2147483648L AS INT))' due to data type mismatch: The accuracy provided must be a positive integer literal (current value = -2147483648); line 1 pos 7 + +select percentile_approx(10.0, 0.5, 4294967297) +10.0 + +select percentile_approx(10.0, 0.5, null) +org.apache.spark.sql.AnalysisException +cannot resolve 'percentile_approx(10.0BD, CAST(0.5BD AS DOUBLE), CAST(NULL AS INT))' due to data type mismatch: The accuracy provided must be a positive integer literal (current value = 0); line 1 pos 7 + +select percentile_approx(10.0, 0.5, 1.2) +10.0 + +select percentile_approx(10.0, null, 1) +scala.MatchError +null + + +select percentile_approx(10.0, array(0.2, 0.4, null), 1) +[10.0,10.0,10.0] ``` ##### behavior after ```sql +select percentile_approx(10.0, 0.5, 2147483648) +10.0 + +select percentile_approx(10.0, 0.5, 4294967297) +10.0 + +select percentile_approx(10.0, 0.5, null) +org.apache.spark.sql.AnalysisException +cannot resolve 'percentile_approx(10.0BD, 0.5BD, NULL)' due to data type mismatch: argument 3 requires integral type, however, 'NULL' is of null type.; line 1 pos 7 + +select percentile_approx(10.0, 0.5, 1.2) +org.apache.spark.sql.AnalysisException +cannot resolve 'percentile_approx(10.0BD, 0.5BD, 1.2BD)' due to data type mismatch: argument 3 requires integral type, however, '1.2BD' is of decimal(2,1) type.; line 1 pos 7 + +select percentile_approx(10.0, null, 1) +java.lang.IllegalArgumentException +The value of percentage must be be between 0.0 and 1.0, but got null + +select percentile_approx(10.0, array(0.2, 0.4, null), 1) +java.lang.IllegalArgumentException +Each value of the percentage array must be be between 0.0 and 1.0, but got [0.2,0.4,null] ``` ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? yes, fix some improper usages of percentile_approx as cases list above ### How was this patch tested? add ut Closes #26905 from yaooqinn/SPARK-30266. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 07:03:26 -05:00
- Since Spark 3.0, the function `percentile_approx` and its alias `approx_percentile` only accept integral value with range in `[1, 2147483647]` as its 3rd argument `accuracy`, fractional and string types are disallowed, e.g. `percentile_approx(10.0, 0.2, 1.8D)` will cause `AnalysisException`. In Spark version 2.4 and earlier, if `accuracy` is fractional or string value, it will be coerced to an int value, `percentile_approx(10.0, 0.2, 1.8D)` is operated as `percentile_approx(10.0, 0.2, 1)` which results in `10.0`.
[SPARK-30507][SQL] TableCalalog reserved properties shoudn't be changed via options or tblpropeties ### What changes were proposed in this pull request? TableCatalog reserves some properties, e,g `provider`, `location` for internal usage. Some of them are static once create, some of them need specific syntax to modify. Instead of using `OPTIONS (k='v')` or TBLPROPERTIES (k='v'), if k is a reserved TableCatalog property, we should use its specific syntax to add/modify/delete it. e.g. `provider` is a reserved property, we should use the `USING` clause to specify it, and should not allow `ALTER TABLE ... UNSET TBLPROPERTIES('provider')` to delete it. Also, there are two paths for v1/v2 catalog tables to resolve these properties, e.g. the v1 session catalog tables will only use the `USING` clause to decide `provider` but v2 tables will also lookup OPTION/TBLPROPERTIES(although there is a bug prohibit it). Additionally, 'path' is not reserved but holds special meaning for `LOCATION` and it is used in `CREATE/REPLACE TABLE`'s `OPTIONS` sub-clause. Now for the session catalog tables, the `path` is case-insensitive, but for the non-session catalog tables, it is case-sensitive, we should make it both case insensitive for disambiguation. ### Why are the changes needed? prevent reserved properties from being modified unexpectedly unify the property resolution for v1/v2. fix some bugs. ### Does this PR introduce any user-facing change? yes 1 . `location` and `provider` (case sensitive) cannot be used in `CREATE/REPLACE TABLE ... OPTIONS/TBLPROPETIES` and `ALTER TABLE ... SET TBLPROPERTIES (...)`, if legacy on, they will be ignored to let the command success without having side effects 3. Once `path` in `CREATE/REPLACE TABLE ... OPTIONS` is case insensitive for v1 but sensitive for v2, but now we change it case insensitive for both kinds of tables, then v2 tables will also fail if `LOCATION` and `OPTIONS('PaTh' ='abc')` are both specified or will pick `PaTh`'s value as table location if `LOCATION` is missing. 4. Now we will detect if there are two different case `path` keys or more in `CREATE/REPLACE TABLE ... OPTIONS`, once it is a kind of unexpected last-win policy for v1, and v2 is case sensitive. ### How was this patch tested? add ut Closes #27197 from yaooqinn/SPARK-30507. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-16 08:46:07 -05:00
- Since Spark 3.0, the properties listing below become reserved, commands will fail if we specify reserved properties in places like `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER TABLE ... SET TBLPROPERTIES`. We need their specific clauses to specify them, e.g. `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. We can set `spark.sql.legacy.property.nonReserved` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, e.g `SET DBPROTERTIES('location'='/tmp')` will affect nothing. In Spark version 2.4 and earlier, these properties are neither reserved nor have side effects, e.g. `SET DBPROTERTIES('location'='/tmp')` will not change the location of the database but only create a headless property just like `'a'='b'`.
<table class="table">
<tr>
<th>
<b>Property(case sensitive)</b>
</th>
<th>
<b>Database Reserved</b>
</th>
<th>
<b>Table Reserved</b>
</th>
<th>
<b>Remarks</b>
</th>
</tr>
<tr>
<td>
provider
</td>
<td>
no
</td>
<td>
yes
</td>
<td>
For tables, please use the USING clause to specify it. Once set, it can't be changed.
</td>
</tr>
<tr>
<td>
location
</td>
<td>
yes
</td>
<td>
yes
</td>
<td>
For databases and tables, please use the LOCATION clause to specify it.
</td>
</tr>
<tr>
<td>
owner
</td>
<td>
yes
</td>
<td>
yes
</td>
<td>
For databases and tables, it is determined by the user who runs spark and create the table.
[SPARK-30507][SQL] TableCalalog reserved properties shoudn't be changed via options or tblpropeties ### What changes were proposed in this pull request? TableCatalog reserves some properties, e,g `provider`, `location` for internal usage. Some of them are static once create, some of them need specific syntax to modify. Instead of using `OPTIONS (k='v')` or TBLPROPERTIES (k='v'), if k is a reserved TableCatalog property, we should use its specific syntax to add/modify/delete it. e.g. `provider` is a reserved property, we should use the `USING` clause to specify it, and should not allow `ALTER TABLE ... UNSET TBLPROPERTIES('provider')` to delete it. Also, there are two paths for v1/v2 catalog tables to resolve these properties, e.g. the v1 session catalog tables will only use the `USING` clause to decide `provider` but v2 tables will also lookup OPTION/TBLPROPERTIES(although there is a bug prohibit it). Additionally, 'path' is not reserved but holds special meaning for `LOCATION` and it is used in `CREATE/REPLACE TABLE`'s `OPTIONS` sub-clause. Now for the session catalog tables, the `path` is case-insensitive, but for the non-session catalog tables, it is case-sensitive, we should make it both case insensitive for disambiguation. ### Why are the changes needed? prevent reserved properties from being modified unexpectedly unify the property resolution for v1/v2. fix some bugs. ### Does this PR introduce any user-facing change? yes 1 . `location` and `provider` (case sensitive) cannot be used in `CREATE/REPLACE TABLE ... OPTIONS/TBLPROPETIES` and `ALTER TABLE ... SET TBLPROPERTIES (...)`, if legacy on, they will be ignored to let the command success without having side effects 3. Once `path` in `CREATE/REPLACE TABLE ... OPTIONS` is case insensitive for v1 but sensitive for v2, but now we change it case insensitive for both kinds of tables, then v2 tables will also fail if `LOCATION` and `OPTIONS('PaTh' ='abc')` are both specified or will pick `PaTh`'s value as table location if `LOCATION` is missing. 4. Now we will detect if there are two different case `path` keys or more in `CREATE/REPLACE TABLE ... OPTIONS`, once it is a kind of unexpected last-win policy for v1, and v2 is case sensitive. ### How was this patch tested? add ut Closes #27197 from yaooqinn/SPARK-30507. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-16 08:46:07 -05:00
</td>
</tr>
</table>
[SPARK-30234][SQL] ADD FILE cannot add directories from sql CLI ### What changes were proposed in this pull request? Now users can add directories from sql CLI as well using ADD FILE command and setting spark.sql.addDirectory.recursive to true. ### Why are the changes needed? In SPARK-4687, support was added for adding directories as resources. But sql users cannot use that feature from CLI. `ADD FILE /path/to/folder` gives the following error: `org.apache.spark.SparkException: Added file /path/to/folder is a directory and recursive is not turned on.` Users need to turn on `recursive` for adding directories. Thus a configuration was required which will allow users to turn on `recursive`. Also Hive allow users to add directories from their shell. ### Does this PR introduce any user-facing change? Yes. Users can set recursive using `spark.sql.addDirectory.recursive`. ### How was this patch tested? Manually. Will add test cases soon. SPARK SCREENSHOTS When `spark.sql.addDirectory.recursive` is not turned on. ![Screenshot from 2019-12-13 08-02-13](https://user-images.githubusercontent.com/15366835/70765124-c6b4a100-1d7f-11ea-9352-9c010af5b38b.png) After setting `spark.sql.addDirectory.recursive` to true. ![Screenshot from 2019-12-13 08-02-59](https://user-images.githubusercontent.com/15366835/70765118-be5c6600-1d7f-11ea-9faf-0b1c46ee299b.png) HIVE SCREENSHOT ![Screenshot from 2019-12-13 14-44-41](https://user-images.githubusercontent.com/15366835/70788979-17e08700-1db8-11ea-9c0c-b6d6f6e80a35.png) `RELEASE_NOTES.txt` is text file while `dummy` is a directory. Closes #26863 from iRakson/SPARK-30234. Lead-authored-by: root1 <raksonrakesh@gmail.com> Co-authored-by: iRakson <raksonrakesh@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 08:36:45 -05:00
- Since Spark 3.0, `ADD FILE` can be used to add file directories as well. Earlier only single files can be added using this command. To restore the behaviour of earlier versions, set `spark.sql.legacy.addDirectory.recursive` to false.
- Since Spark 3.0, `SHOW TBLPROPERTIES` will cause `AnalysisException` if the table does not exist. In Spark version 2.4 and earlier, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view will cause `AnalysisException`. In Spark version 2.4 and earlier, it returned an empty result.
## Upgrading from Spark SQL 2.4.4 to 2.4.5
- Since Spark 2.4.5, `TRUNCATE TABLE` command tries to set back original permission and ACLs during re-creating the table/partition paths. To restore the behaviour of earlier versions, set `spark.sql.truncateTable.ignorePermissionAcl.enabled` to `true`.
- Since Spark 2.4.5, `spark.sql.legacy.mssqlserver.numericMapping.enabled` configuration is added in order to support the legacy MsSQLServer dialect mapping behavior using IntegerType and DoubleType for SMALLINT and REAL JDBC types, respectively. To restore the behaviour of 2.4.3 and earlier versions, set `spark.sql.legacy.mssqlserver.numericMapping.enabled` to `true`.
## Upgrading from Spark SQL 2.4.3 to 2.4.4
- Since Spark 2.4.4, according to [MsSqlServer Guide](https://docs.microsoft.com/en-us/sql/connect/jdbc/using-basic-data-types?view=sql-server-2017), MsSQLServer JDBC Dialect uses ShortType and FloatType for SMALLINT and REAL, respectively. Previously, IntegerType and DoubleType is used.
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
## 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.
<table class="table">
<tr>
<th>
<b>Query</b>
</th>
<th>
<b>Spark 2.3 or Prior</b>
</th>
<th>
<b>Spark 2.4</b>
</th>
<th>
<b>Remarks</b>
</th>
</tr>
<tr>
<td>
<code>SELECT array_contains(array(1), 1.34D);</code>
</td>
<td>
<code>true</code>
</td>
<td>
<code>false</code>
</td>
<td>
In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively.
</td>
</tr>
<tr>
<td>
<code>SELECT array_contains(array(1), '1');</code>
</td>
<td>
<code>true</code>
</td>
<td>
<code>AnalysisException</code> is thrown.
</td>
<td>
Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, <code>AnalysisException</code> is thrown since integer type can not be promoted to string type in a loss-less manner.
</td>
</tr>
<tr>
<td>
<code>SELECT array_contains(array(1), 'anystring');</code>
</td>
<td>
<code>null</code>
</td>
<td>
<code>AnalysisException</code> is thrown.
</td>
<td>
Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, <code>AnalysisException</code> is thrown since integer type can not be promoted to string type in a loss-less manner.
</td>
</tr>
</table>
- 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:
<table class="table">
<tr>
<th>
<b>InputA \ InputB</b>
</th>
<th>
<b>NullType</b>
</th>
<th>
<b>IntegerType</b>
</th>
<th>
<b>LongType</b>
</th>
<th>
<b>DecimalType(38,0)*</b>
</th>
<th>
<b>DoubleType</b>
</th>
<th>
<b>DateType</b>
</th>
<th>
<b>TimestampType</b>
</th>
<th>
<b>StringType</b>
</th>
</tr>
<tr>
<td>
<b>NullType</b>
</td>
<td>NullType</td>
<td>IntegerType</td>
<td>LongType</td>
<td>DecimalType(38,0)</td>
<td>DoubleType</td>
<td>DateType</td>
<td>TimestampType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>IntegerType</b>
</td>
<td>IntegerType</td>
<td>IntegerType</td>
<td>LongType</td>
<td>DecimalType(38,0)</td>
<td>DoubleType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>LongType</b>
</td>
<td>LongType</td>
<td>LongType</td>
<td>LongType</td>
<td>DecimalType(38,0)</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>DecimalType(38,0)*</b>
</td>
<td>DecimalType(38,0)</td>
<td>DecimalType(38,0)</td>
<td>DecimalType(38,0)</td>
<td>DecimalType(38,0)</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>DoubleType</b>
</td>
<td>DoubleType</td>
<td>DoubleType</td>
<td>StringType</td>
<td>StringType</td>
<td>DoubleType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>DateType</b>
</td>
<td>DateType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>DateType</td>
<td>TimestampType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>TimestampType</b>
</td>
<td>TimestampType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>TimestampType</td>
<td>TimestampType</td>
<td>StringType</td>
</tr>
<tr>
<td>
<b>StringType</b>
</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
<td>StringType</td>
</tr>
</table>
Note that, for <b>DecimalType(38,0)*</b>, 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 [Join Strategy Hints for SQL Queries](sql-performance-tuning.html#join-strategy-hints-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489).
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
- 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<Row>`. 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` (
<a href="api/scala/index.html#org.apache.spark.sql.SQLContext@read:DataFrameReader">Scala</a>,
<a href="api/java/org/apache/spark/sql/SQLContext.html#read()">Java</a>,
<a href="api/python/pyspark.sql.html#pyspark.sql.SQLContext.read">Python</a>
) and `DataFrame.write` (
<a href="api/scala/index.html#org.apache.spark.sql.DataFrame@write:DataFrameWriter">Scala</a>,
<a href="api/java/org/apache/spark/sql/Dataset.html#write()">Java</a>,
<a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame.write">Python</a>
) 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`.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% 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 %}
</div>
<div data-lang="java" markdown="1">
{% 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 %}
</div>
<div data-lang="python" markdown="1">
{% 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 %}
</div>
</div>
#### 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`.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
sqlContext.udf.register("strLen", (s: String) => s.length())
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType);
{% endhighlight %}
</div>
</div>
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 built-in Hive,
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
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)
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
* 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`
* `CREATE TABLE LIKE`
[SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation ### 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 <gurwls223@apache.org> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-15 14:17:30 -04:00
* `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.