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

1093 lines
88 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 3.1 to 3.2
- Since Spark 3.2, ADD FILE/JAR/ARCHIVE commands require each path to be enclosed by `"` or `'` if the path contains whitespaces.
- Since Spark 3.2, all the supported JDBC dialects use StringType for ROWID. In Spark 3.1 or earlier, Oracle dialect uses StringType and the other dialects use LongType.
[SPARK-34333][SQL] Fix PostgresDialect to handle money types properly ### What changes were proposed in this pull request? This PR changes the type mapping for `money` and `money[]` types for PostgreSQL. Currently, those types are tried to convert to `DoubleType` and `ArrayType` of `double` respectively. But the JDBC driver seems not to be able to handle those types properly. https://github.com/pgjdbc/pgjdbc/issues/100 https://github.com/pgjdbc/pgjdbc/issues/1405 Due to these issue, we can get the error like as follows. money type. ``` [info] org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0) (192.168.1.204 executor driver): org.postgresql.util.PSQLException: Bad value for type double : 1,000.00 [info] at org.postgresql.jdbc.PgResultSet.toDouble(PgResultSet.java:3104) [info] at org.postgresql.jdbc.PgResultSet.getDouble(PgResultSet.java:2432) [info] at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeGetter$5(JdbcUtils.scala:418) ``` money[] type. ``` [info] org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0) (192.168.1.204 executor driver): org.postgresql.util.PSQLException: Bad value for type double : $2,000.00 [info] at org.postgresql.jdbc.PgResultSet.toDouble(PgResultSet.java:3104) [info] at org.postgresql.jdbc.ArrayDecoding$5.parseValue(ArrayDecoding.java:235) [info] at org.postgresql.jdbc.ArrayDecoding$AbstractObjectStringArrayDecoder.populateFromString(ArrayDecoding.java:122) [info] at org.postgresql.jdbc.ArrayDecoding.readStringArray(ArrayDecoding.java:764) [info] at org.postgresql.jdbc.PgArray.buildArray(PgArray.java:310) [info] at org.postgresql.jdbc.PgArray.getArrayImpl(PgArray.java:171) [info] at org.postgresql.jdbc.PgArray.getArray(PgArray.java:111) ``` For money type, a known workaround is to treat it as string so this PR do it. For money[], however, there is no reasonable workaround so this PR remove the support. ### Why are the changes needed? This is a bug. ### Does this PR introduce _any_ user-facing change? Yes. As of this PR merged, money type is mapped to `StringType` rather than `DoubleType` and the support for money[] is stopped. For money type, if the value is less than one thousand, `$100.00` for instance, it works without this change so I also updated the migration guide because it's a behavior change for such small values. On the other hand, money[] seems not to work with any value but mentioned in the migration guide just in case. ### How was this patch tested? New test. Closes #31442 from sarutak/fix-for-money-type. Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com> Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-02-16 20:50:06 -05:00
- In Spark 3.2, PostgreSQL JDBC dialect uses StringType for MONEY and MONEY[] is not supported due to the JDBC driver for PostgreSQL can't handle those types properly. In Spark 3.1 or earlier, DoubleType and ArrayType of DoubleType are used respectively.
- In Spark 3.2, `spark.sql.adaptive.enabled` is enabled by default. To restore the behavior before Spark 3.2, you can set `spark.sql.adaptive.enabled` to `false`.
- In Spark 3.2, the following meta-characters are escaped in the `show()` action. In Spark 3.1 or earlier, the following metacharacters are output as it is.
* `\n` (new line)
* `\r` (carrige ret)
* `\t` (horizontal tab)
* `\f` (form feed)
* `\b` (backspace)
* `\u000B` (vertical tab)
* `\u0007` (bell)
- In Spark 3.2, `ALTER TABLE .. RENAME TO PARTITION` throws `PartitionAlreadyExistsException` instead of `AnalysisException` for tables from Hive external when the target partition already exists.
- In Spark 3.2, script transform default FIELD DELIMIT is `\u0001` for no serde mode, serde property `field.delim` is `\t` for Hive serde mode when user specifies serde. In Spark 3.1 or earlier, the default FIELD DELIMIT is `\t`, serde property `field.delim` is `\u0001` for Hive serde mode when user specifies serde.
[SPARK-33930][SQL] Script Transform default FIELD DELIMIT should be \u0001 for no serde ### What changes were proposed in this pull request? For same SQL ``` SELECT TRANSFORM(a, b, c, null) ROW FORMAT DELIMITED USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY '&' FROM (select 1 as a, 2 as b, 3 as c) t ``` In hive: ``` hive> SELECT TRANSFORM(a, b, c, null) > ROW FORMAT DELIMITED > USING 'cat' > ROW FORMAT DELIMITED > FIELDS TERMINATED BY '&' > FROM (select 1 as a, 2 as b, 3 as c) t; OK 123\N NULL Time taken: 14.519 seconds, Fetched: 1 row(s) hive> packet_write_wait: Connection to 10.191.58.100 port 32200: Broken pipe ``` In Spark ``` Spark master: local[*], Application Id: local-1609225830376 spark-sql> SELECT TRANSFORM(a, b, c, null) > ROW FORMAT DELIMITED > USING 'cat' > ROW FORMAT DELIMITED > FIELDS TERMINATED BY '&' > FROM (select 1 as a, 2 as b, 3 as c) t; 1 2 3 null NULL Time taken: 4.297 seconds, Fetched 1 row(s) spark-sql> ``` We should keep same. Change default ROW FORMAT FIELD DELIMIT to `\u0001` In hive default value is '1' to char is '\u0001' ``` bucket_count -1 column.name.delimiter , columns columns.comments columns.types file.inputformat org.apache.hadoop.hive.ql.io.NullRowsInputFormat ``` ### Why are the changes needed? Keep same behavior with hive ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added UT Closes #30958 from AngersZhuuuu/SPARK-33930. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-29 09:26:27 -05:00
- In Spark 3.2, the auto-generated `Cast` (such as those added by type coercion rules) will be stripped when generating column alias names. E.g., `sql("SELECT floor(1)").columns` will be `FLOOR(1)` instead of `FLOOR(CAST(1 AS DOUBLE))`.
[SPARK-34157][SQL] Unify output of SHOW TABLES and pass output attributes properly ### What changes were proposed in this pull request? The current implement of some DDL not unify the output and not pass the output properly to physical command. Such as: The `ShowTables` output attributes `namespace`, but `ShowTablesCommand` output attributes `database`. As the query plan, this PR pass the output attributes from `ShowTables` to `ShowTablesCommand`, `ShowTableExtended ` to `ShowTablesCommand`. Take `show tables` and `show table extended like 'tbl'` as example. The output before this PR: `show tables` |database|tableName|isTemporary| -- | -- | -- | default| tbl| false| If catalog is v2 session catalog, the output before this PR: |namespace|tableName| -- | -- | default| tbl `show table extended like 'tbl'` |database|tableName|isTemporary| information| -- | -- | -- | -- | default| tbl| false|Database: default...| The output after this PR: `show tables` |namespace|tableName|isTemporary| -- | -- | -- | default| tbl| false| `show table extended like 'tbl'` |namespace|tableName|isTemporary| information| -- | -- | -- | -- | default| tbl| false|Database: default...| ### Why are the changes needed? This PR have benefits as follows: First, Unify schema for the output of SHOW TABLES. Second, pass the output attributes could keep the expr ID unchanged, so that avoid bugs when we apply more operators above the command output dataframe. ### Does this PR introduce _any_ user-facing change? Yes. The output schema of `SHOW TABLES` replace `database` by `namespace`. ### How was this patch tested? Jenkins test. Closes #31245 from beliefer/SPARK-34157. Lead-authored-by: gengjiaan <gengjiaan@360.cn> Co-authored-by: beliefer <beliefer@163.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 03:39:58 -05:00
- In Spark 3.2, the output schema of `SHOW TABLES` becomes `namespace: string, tableName: string, isTemporary: boolean`. In Spark 3.1 or earlier, the `namespace` field was named `database` for the builtin catalog, and there is no `isTemporary` field for v2 catalogs. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.
- In Spark 3.2, the output schema of `SHOW TABLE EXTENDED` becomes `namespace: string, tableName: string, isTemporary: boolean, information: string`. In Spark 3.1 or earlier, the `namespace` field was named `database` for the builtin catalog, and no change for the v2 catalogs. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.
- In Spark 3.2, the output schema of `SHOW TBLPROPERTIES` becomes `key: string, value: string` whether you specify the table property key or not. In Spark 3.1 and earlier, the output schema of `SHOW TBLPROPERTIES` is `value: string` when you specify the table property key. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.
- In Spark 3.2, the output schema of `DESCRIBE NAMESPACE` becomes `info_name: string, info_value: string`. In Spark 3.1 or earlier, the `info_name` field was named `database_description_item` and the `info_value` field was named `database_description_value` for the builtin catalog. To restore the old schema with the builtin catalog, you can set `spark.sql.legacy.keepCommandOutputSchema` to `true`.
[SPARK-34138][SQL] Keep dependants cached while refreshing v1 tables ### What changes were proposed in this pull request? This PR changes cache refreshing of v1 tables in v1 commands. In particular, v1 table dependents are not removed from the cache after this PR. Comparing to current implementation, we just clear cached data of all dependents and keep them in the cache. So, the next actions will fill in the cached data of the original v1 table and its dependents. In more details: 1. Modified the `CatalogImpl.refreshTable()` method to use `recacheByPlan()` instead of `lookupCachedData()`, `uncacheQuery()` and `cacheQuery()`. Users can call this method via public API like `spark.catalog.refreshTable()`. 2. Rewritten the part in `CatalogImpl.refreshTable()` which was responsible for table meta-data refreshing because this code stopped to work properly after removing of the second `sparkSession.table(tableIdent)`. 3. Added new private method `invalidateCachedTable()` to `SessionCatalog`. Comparing to the existing `SessionCatalog.refreshTable`, it invalidates the relation cache only. If we called `SessionCatalog.refreshTable` from `CatalogImpl.refreshTable()`, we would refresh temporary and global temporary views twice (that could lead to refreshing file index twice). ### Why are the changes needed? 1. This should improve user experience with table/view caching. For example, let's imagine that an user has cached v1 table and cached view based on the table. And the user passed the table to external library which drops/renames/adds partitions in the v1 table. Unfortunately, the user gets the view uncached after that even he/she hasn't uncached the view explicitly. 2. To improve code maintenance. 3. To reduce the amount of calls to Hive external catalog. 4. Also this should speed up table recaching. 5. To have the same behavior as for v2 tables supported by https://github.com/apache/spark/pull/31172 ### Does this PR introduce _any_ user-facing change? From the view of the correctness of query results, there are no behavior changes but the changes might influence on consuming memory and query execution time. For example: Before: ```scala scala> sql("CREATE TABLE tbl (c int)") scala> sql("CACHE TABLE tbl") scala> sql("CREATE VIEW v AS SELECT * FROM tbl") scala> sql("CACHE TABLE v") scala> spark.catalog.isCached("v") res6: Boolean = true scala> spark.catalog.refreshTable("tbl") scala> spark.catalog.isCached("v") res8: Boolean = false ``` After: ```scala scala> spark.catalog.refreshTable("tbl") scala> spark.catalog.isCached("v") res8: Boolean = true ``` ### How was this patch tested? 1. Added new unit tests that create a view, a temporary view and a global temporary view on top of v1/v2 tables, and refresh the base table via `ALTER TABLE .. ADD/DROP/RENAME PARTITION`. 2. By running the unified test suites: ``` $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite" $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite" # build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite" ``` Closes #31206 from MaxGekk/refreshTable-recache-by-plan. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 08:03:24 -05:00
- In Spark 3.2, table refreshing clears cached data of the table as well as of all its dependents such as views while keeping the dependents cached. The following commands perform table refreshing:
* `ALTER TABLE .. ADD PARTITION`
* `ALTER TABLE .. RENAME PARTITION`
* `ALTER TABLE .. DROP PARTITION`
* `ALTER TABLE .. RECOVER PARTITIONS`
* `MSCK REPAIR TABLE`
* `LOAD DATA`
[SPARK-34138][SQL] Keep dependants cached while refreshing v1 tables ### What changes were proposed in this pull request? This PR changes cache refreshing of v1 tables in v1 commands. In particular, v1 table dependents are not removed from the cache after this PR. Comparing to current implementation, we just clear cached data of all dependents and keep them in the cache. So, the next actions will fill in the cached data of the original v1 table and its dependents. In more details: 1. Modified the `CatalogImpl.refreshTable()` method to use `recacheByPlan()` instead of `lookupCachedData()`, `uncacheQuery()` and `cacheQuery()`. Users can call this method via public API like `spark.catalog.refreshTable()`. 2. Rewritten the part in `CatalogImpl.refreshTable()` which was responsible for table meta-data refreshing because this code stopped to work properly after removing of the second `sparkSession.table(tableIdent)`. 3. Added new private method `invalidateCachedTable()` to `SessionCatalog`. Comparing to the existing `SessionCatalog.refreshTable`, it invalidates the relation cache only. If we called `SessionCatalog.refreshTable` from `CatalogImpl.refreshTable()`, we would refresh temporary and global temporary views twice (that could lead to refreshing file index twice). ### Why are the changes needed? 1. This should improve user experience with table/view caching. For example, let's imagine that an user has cached v1 table and cached view based on the table. And the user passed the table to external library which drops/renames/adds partitions in the v1 table. Unfortunately, the user gets the view uncached after that even he/she hasn't uncached the view explicitly. 2. To improve code maintenance. 3. To reduce the amount of calls to Hive external catalog. 4. Also this should speed up table recaching. 5. To have the same behavior as for v2 tables supported by https://github.com/apache/spark/pull/31172 ### Does this PR introduce _any_ user-facing change? From the view of the correctness of query results, there are no behavior changes but the changes might influence on consuming memory and query execution time. For example: Before: ```scala scala> sql("CREATE TABLE tbl (c int)") scala> sql("CACHE TABLE tbl") scala> sql("CREATE VIEW v AS SELECT * FROM tbl") scala> sql("CACHE TABLE v") scala> spark.catalog.isCached("v") res6: Boolean = true scala> spark.catalog.refreshTable("tbl") scala> spark.catalog.isCached("v") res8: Boolean = false ``` After: ```scala scala> spark.catalog.refreshTable("tbl") scala> spark.catalog.isCached("v") res8: Boolean = true ``` ### How was this patch tested? 1. Added new unit tests that create a view, a temporary view and a global temporary view on top of v1/v2 tables, and refresh the base table via `ALTER TABLE .. ADD/DROP/RENAME PARTITION`. 2. By running the unified test suites: ``` $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite" $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite" # build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite" ``` Closes #31206 from MaxGekk/refreshTable-recache-by-plan. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 08:03:24 -05:00
* `REFRESH TABLE`
[SPARK-34215][SQL] Keep tables cached after truncation ### What changes were proposed in this pull request? Invoke `CatalogImpl.refreshTable()` instead of combination of `SessionCatalog.refreshTable()` + `uncacheQuery()`. This allows to clear cached table data while keeping the table cached. ### Why are the changes needed? 1. To improve user experience with Spark SQL 2. To be consistent to other commands, see https://github.com/apache/spark/pull/31206 ### Does this PR introduce _any_ user-facing change? Yes. Before: ```scala scala> sql("CREATE TABLE tbl (c0 int)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("INSERT INTO tbl SELECT 0") res2: org.apache.spark.sql.DataFrame = [] scala> sql("CACHE TABLE tbl") res3: org.apache.spark.sql.DataFrame = [] scala> sql("SELECT * FROM tbl").show(false) +---+ |c0 | +---+ |0 | +---+ scala> spark.catalog.isCached("tbl") res5: Boolean = true scala> sql("TRUNCATE TABLE tbl") res6: org.apache.spark.sql.DataFrame = [] scala> spark.catalog.isCached("tbl") res7: Boolean = false ``` After: ```scala scala> sql("TRUNCATE TABLE tbl") res6: org.apache.spark.sql.DataFrame = [] scala> spark.catalog.isCached("tbl") res7: Boolean = true ``` ### How was this patch tested? Added new test to `CachedTableSuite`: ``` $ build/sbt -Phive -Phive-thriftserver "test:testOnly *CachedTableSuite" $ build/sbt -Phive -Phive-thriftserver "test:testOnly *CatalogedDDLSuite" ``` Closes #31308 from MaxGekk/truncate-table-cached. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-26 10:36:44 -05:00
* `TRUNCATE TABLE`
[SPARK-34138][SQL] Keep dependants cached while refreshing v1 tables ### What changes were proposed in this pull request? This PR changes cache refreshing of v1 tables in v1 commands. In particular, v1 table dependents are not removed from the cache after this PR. Comparing to current implementation, we just clear cached data of all dependents and keep them in the cache. So, the next actions will fill in the cached data of the original v1 table and its dependents. In more details: 1. Modified the `CatalogImpl.refreshTable()` method to use `recacheByPlan()` instead of `lookupCachedData()`, `uncacheQuery()` and `cacheQuery()`. Users can call this method via public API like `spark.catalog.refreshTable()`. 2. Rewritten the part in `CatalogImpl.refreshTable()` which was responsible for table meta-data refreshing because this code stopped to work properly after removing of the second `sparkSession.table(tableIdent)`. 3. Added new private method `invalidateCachedTable()` to `SessionCatalog`. Comparing to the existing `SessionCatalog.refreshTable`, it invalidates the relation cache only. If we called `SessionCatalog.refreshTable` from `CatalogImpl.refreshTable()`, we would refresh temporary and global temporary views twice (that could lead to refreshing file index twice). ### Why are the changes needed? 1. This should improve user experience with table/view caching. For example, let's imagine that an user has cached v1 table and cached view based on the table. And the user passed the table to external library which drops/renames/adds partitions in the v1 table. Unfortunately, the user gets the view uncached after that even he/she hasn't uncached the view explicitly. 2. To improve code maintenance. 3. To reduce the amount of calls to Hive external catalog. 4. Also this should speed up table recaching. 5. To have the same behavior as for v2 tables supported by https://github.com/apache/spark/pull/31172 ### Does this PR introduce _any_ user-facing change? From the view of the correctness of query results, there are no behavior changes but the changes might influence on consuming memory and query execution time. For example: Before: ```scala scala> sql("CREATE TABLE tbl (c int)") scala> sql("CACHE TABLE tbl") scala> sql("CREATE VIEW v AS SELECT * FROM tbl") scala> sql("CACHE TABLE v") scala> spark.catalog.isCached("v") res6: Boolean = true scala> spark.catalog.refreshTable("tbl") scala> spark.catalog.isCached("v") res8: Boolean = false ``` After: ```scala scala> spark.catalog.refreshTable("tbl") scala> spark.catalog.isCached("v") res8: Boolean = true ``` ### How was this patch tested? 1. Added new unit tests that create a view, a temporary view and a global temporary view on top of v1/v2 tables, and refresh the base table via `ALTER TABLE .. ADD/DROP/RENAME PARTITION`. 2. By running the unified test suites: ``` $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableAddPartitionSuite" $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableDropPartitionSuite" # build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite" ``` Closes #31206 from MaxGekk/refreshTable-recache-by-plan. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-01-21 08:03:24 -05:00
* and the method `spark.catalog.refreshTable`
In Spark 3.1 and earlier, table refreshing leaves dependents uncached.
- In Spark 3.2, the usage of `count(tblName.*)` is blocked to avoid producing ambiguous results. Because `count(*)` and `count(tblName.*)` will output differently if there is any null values. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.allowStarWithSingleTableIdentifierInCount` to `true`.
- In Spark 3.2, we support typed literals in the partition spec of INSERT and ADD/DROP/RENAME PARTITION. For example, `ADD PARTITION(dt = date'2020-01-01')` adds a partition with date value `2020-01-01`. In Spark 3.1 and earlier, the partition value will be parsed as string value `date '2020-01-01'`, which is an illegal date value, and we add a partition with null value at the end.
- In Spark 3.2, `DataFrameNaFunctions.replace()` no longer uses exact string match for the input column names, to match the SQL syntax and support qualified column names. Input column name having a dot in the name (not nested) needs to be escaped with backtick \`. Now, it throws `AnalysisException` if the column is not found in the data frame schema. It also throws `IllegalArgumentException` if the input column name is a nested column. In Spark 3.1 and earlier, it used to ignore invalid input column name and nested column name.
- In Spark 3.2, the dates subtraction expression such as `date1 - date2` returns values of `DayTimeIntervalType`. In Spark 3.1 and earlier, the returned type is `CalendarIntervalType`. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.interval.enabled` to `true`.
- In Spark 3.2, the timestamps subtraction expression such as `timestamp '2021-03-31 23:48:00' - timestamp '2021-01-01 00:00:00'` returns values of `DayTimeIntervalType`. In Spark 3.1 and earlier, the type of the same expression is `CalendarIntervalType`. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.interval.enabled` to `true`.
- In Spark 3.2, `CREATE TABLE .. LIKE ..` command can not use reserved properties. You need their specific clauses to specify them, for example, `CREATE TABLE test1 LIKE test LOCATION 'some path'`. You can set `spark.sql.legacy.notReserveProperties` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, for example: `TBLPROPERTIES('owner'='yao')` will have no effect. In Spark version 3.1 and below, the reserved properties can be used in `CREATE TABLE .. LIKE ..` command but have no side effects, for example, `TBLPROPERTIES('location'='/tmp')` does not change the location of the table but only create a headless property just like `'a'='b'`.
- In Spark 3.2, `TRANSFORM` operator can't support alias in inputs. In Spark 3.1 and earlier, we can write script transform like `SELECT TRANSFORM(a AS c1, b AS c2) USING 'cat' FROM TBL`.
- In Spark 3.2, `TRANSFORM` operator can support `ArrayType/MapType/StructType` without Hive SerDe, in this mode, we use `StructsToJosn` to convert `ArrayType/MapType/StructType` column to `STRING` and use `JsonToStructs` to parse `STRING` to `ArrayType/MapType/StructType`. In Spark 3.1, Spark just support case `ArrayType/MapType/StructType` column as `STRING` but can't support parse `STRING` to `ArrayType/MapType/StructType` output columns.
[SPARK-35107][SQL] Parse unit-to-unit interval literals to ANSI intervals ### What changes were proposed in this pull request? Parse the year-month interval literals like `INTERVAL '1-1' YEAR TO MONTH` to values of `YearMonthIntervalType`, and day-time interval literals to `DayTimeIntervalType` values. Currently, Spark SQL supports: - DAY TO HOUR - DAY TO MINUTE - DAY TO SECOND - HOUR TO MINUTE - HOUR TO SECOND - MINUTE TO SECOND All such interval literals are converted to `DayTimeIntervalType`, and `YEAR TO MONTH` to `YearMonthIntervalType` while loosing info about `from` and `to` units. **Note**: new behavior is under the SQL config `spark.sql.legacy.interval.enabled` which is `false` by default. When the config is set to `true`, the interval literals are parsed to `CaledarIntervalType` values. Closes #32176 ### Why are the changes needed? To conform the ANSI SQL standard which assumes conversions of interval literals to year-month or day-time interval but not to mixed interval type like Catalyst's `CalendarIntervalType`. ### Does this PR introduce _any_ user-facing change? Yes. Before: ```sql spark-sql> SELECT INTERVAL '1 01:02:03.123' DAY TO SECOND; 1 days 1 hours 2 minutes 3.123 seconds spark-sql> SELECT typeof(INTERVAL '1 01:02:03.123' DAY TO SECOND); interval ``` After: ```sql spark-sql> SELECT INTERVAL '1 01:02:03.123' DAY TO SECOND; 1 01:02:03.123000000 spark-sql> SELECT typeof(INTERVAL '1 01:02:03.123' DAY TO SECOND); day-time interval ``` ### How was this patch tested? 1. By running the affected test suites: ``` $ ./build/sbt "test:testOnly *.ExpressionParserSuite" $ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql" $ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z create_view.sql" $ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql" $ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql" ``` 2. PostgresSQL tests are executed with `spark.sql.legacy.interval.enabled` is set to `true` to keep compatibility with PostgreSQL output: ```sql > SELECT interval '999' second; 0 years 0 mons 0 days 0 hours 16 mins 39.00 secs ``` Closes #32209 from MaxGekk/parse-ansi-interval-literals. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-19 09:00:59 -04:00
- In Spark 3.2, the unit-to-unit interval literals like `INTERVAL '1-1' YEAR TO MONTH` are converted to ANSI interval types: `YearMonthIntervalType` or `DayTimeIntervalType`. In Spark 3.1 and earlier, such interval literals are converted to `CalendarIntervalType`. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.interval.enabled` to `true`.
- In Spark 3.2, Spark supports `DayTimeIntervalType` and `YearMonthIntervalType` as inputs and outputs of `TRANSFORM` clause in Hive `SERDE` mode, the behavior is different between Hive `SERDE` mode and `ROW FORMAT DELIMITED` mode when these two types are used as inputs. In Hive `SERDE` mode, `DayTimeIntervalType` column is converted to `HiveIntervalDayTime`, its string format is `[-]?d h:m:s.n`, but in `ROW FORMAT DELIMITED` mode the format is `INTERVAL '[-]?d h:m:s.n' DAY TO TIME`. In Hive `SERDE` mode, `YearMonthIntervalType` column is converted to `HiveIntervalYearMonth`, its string format is `[-]?y-m`, but in `ROW FORMAT DELIMITED` mode the format is `INTERVAL '[-]?y-m' YEAR TO MONTH`.
[SPARK-35207][SQL] Normalize hash function behavior with negative zero (floating point types) ### What changes were proposed in this pull request? Generally, we would expect that x = y => hash( x ) = hash( y ). However +-0 hash to different values for floating point types. ``` scala> spark.sql("select hash(cast('0.0' as double)), hash(cast('-0.0' as double))").show +-------------------------+--------------------------+ |hash(CAST(0.0 AS DOUBLE))|hash(CAST(-0.0 AS DOUBLE))| +-------------------------+--------------------------+ | -1670924195| -853646085| +-------------------------+--------------------------+ scala> spark.sql("select cast('0.0' as double) == cast('-0.0' as double)").show +--------------------------------------------+ |(CAST(0.0 AS DOUBLE) = CAST(-0.0 AS DOUBLE))| +--------------------------------------------+ | true| +--------------------------------------------+ ``` Here is an extract from IEEE 754: > The two zeros are distinguishable arithmetically only by either division-byzero ( producing appropriately signed infinities ) or else by the CopySign function recommended by IEEE 754 /854. Infinities, SNaNs, NaNs and Subnormal numbers necessitate four more special cases From this, I deduce that the hash function must produce the same result for 0 and -0. ### Why are the changes needed? It is a correctness issue ### Does this PR introduce _any_ user-facing change? This changes only affect to the hash function applied to -0 value in float and double types ### How was this patch tested? Unit testing and manual testing Closes #32496 from planga82/feature/spark35207_hashnegativezero. Authored-by: Pablo Langa <soypab@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-14 00:40:36 -04:00
- In Spark 3.2, `hash(0) == hash(-0)` for floating point types. Previously, different values were generated.
[SPARK-28551][SQL] CTAS with LOCATION should not allow to a non-empty directory ### What changes were proposed in this pull request? CTAS with location clause acts as an insert overwrite. This can cause problems when there are subdirectories within a location directory. This causes some users to accidentally wipe out directories with very important data. We should not allow CTAS with location to a non-empty directory. ### Why are the changes needed? Hive already handled this scenario: HIVE-11319 Steps to reproduce: ```scala sql("""create external table `demo_CTAS`( `comment` string) PARTITIONED BY (`col1` string, `col2` string) STORED AS parquet location '/tmp/u1/demo_CTAS'""") sql("""INSERT OVERWRITE TABLE demo_CTAS partition (col1='1',col2='1') VALUES ('abc')""") sql("select* from demo_CTAS").show sql("""create table ctas1 location '/tmp/u2/ctas1' as select * from demo_CTAS""") sql("select* from ctas1").show sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""") ``` Before the fix: Both create table operations will succeed. But values in table ctas1 will be replaced by ctas2 accidentally. After the fix: `create table ctas2...` will throw `AnalysisException`: ``` org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true. ``` ### Does this PR introduce _any_ user-facing change? Yes, if the location directory is not empty, CTAS with location will throw AnalysisException ``` sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""") ``` ``` org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true. ``` `CREATE TABLE AS SELECT` with non-empty `LOCATION` will throw `AnalysisException`. To restore the behavior before Spark 3.2, need to set `spark.sql.legacy.allowNonEmptyLocationInCTAS` to `true`. , default value is `false`. Updated SQL migration guide. ### How was this patch tested? Test case added in SQLQuerySuite.scala Closes #32411 from vinodkc/br_fixCTAS_nonempty_dir. Authored-by: Vinod KC <vinod.kc.in@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-20 02:13:18 -04:00
- In Spark 3.2, `CREATE TABLE AS SELECT` with non-empty `LOCATION` will throw `AnalysisException`. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.allowNonEmptyLocationInCTAS` to `true`.
[SPARK-30279][SQL] Support 32 or more grouping attributes for GROUPING_ID ### What changes were proposed in this pull request? This pr intends to support 32 or more grouping attributes for GROUPING_ID. In the current master, an integer overflow can occur to compute grouping IDs; https://github.com/apache/spark/blob/e75d9afb2f282ce79c9fd8bce031287739326a4f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L613 For example, the query below generates wrong grouping IDs in the master; ``` scala> val numCols = 32 // or, 31 scala> val cols = (0 until numCols).map { i => s"c$i" } scala> sql(s"create table test_$numCols (${cols.map(c => s"$c int").mkString(",")}, v int) using parquet") scala> val insertVals = (0 until numCols).map { _ => 1 }.mkString(",") scala> sql(s"insert into test_$numCols values ($insertVals,3)") scala> sql(s"select grouping_id(), sum(v) from test_$numCols group by grouping sets ((${cols.mkString(",")}), (${cols.init.mkString(",")}))").show(10, false) scala> sql(s"drop table test_$numCols") // numCols = 32 +-------------+------+ |grouping_id()|sum(v)| +-------------+------+ |0 |3 | |0 |3 | // Wrong Grouping ID +-------------+------+ // numCols = 31 +-------------+------+ |grouping_id()|sum(v)| +-------------+------+ |0 |3 | |1 |3 | +-------------+------+ ``` To fix this issue, this pr change code to use long values for `GROUPING_ID` instead of int values. ### Why are the changes needed? To support more cases in `GROUPING_ID`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added unit tests. Closes #26918 from maropu/FixGroupingIdIssue. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-06 02:57:03 -05:00
## Upgrading from Spark SQL 3.0 to 3.1
[SPARK-13860][SQL] Change statistical aggregate function to return null instead of Double.NaN when divideByZero ### What changes were proposed in this pull request? As [SPARK-13860](https://issues.apache.org/jira/browse/SPARK-13860) stated, TPCDS Query 39 returns wrong results using SparkSQL. The root cause is that when stddev_samp is applied to a single element set, with TPCDS answer, it return null; as in SparkSQL, it return Double.NaN which caused the wrong result. Add an extra legacy config to fallback into the NaN logical, and return null by default to align with TPCDS standard. ### Why are the changes needed? SQL correctness issue. ### Does this PR introduce any user-facing change? Yes. See sql-migration-guide In Spark 3.1, statistical aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis`, `covar_samp`, `corr` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.statisticalAggregate` to `true`. ### How was this patch tested? Updated DataFrameAggregateSuite/DataFrameWindowFunctionsSuite to test both default and legacy behavior. Adjust DataFrameWindowFunctionsSuite/SQLQueryTestSuite and some R case to update to the default return null behavior. Closes #29983 from leanken/leanken-SPARK-13860. Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-13 09:21:45 -04:00
- In Spark 3.1, statistical aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis`, `covar_samp`, `corr` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.statisticalAggregate` to `true`.
- In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.integerGroupingId` to `true`.
[SPARK-30279][SQL] Support 32 or more grouping attributes for GROUPING_ID ### What changes were proposed in this pull request? This pr intends to support 32 or more grouping attributes for GROUPING_ID. In the current master, an integer overflow can occur to compute grouping IDs; https://github.com/apache/spark/blob/e75d9afb2f282ce79c9fd8bce031287739326a4f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L613 For example, the query below generates wrong grouping IDs in the master; ``` scala> val numCols = 32 // or, 31 scala> val cols = (0 until numCols).map { i => s"c$i" } scala> sql(s"create table test_$numCols (${cols.map(c => s"$c int").mkString(",")}, v int) using parquet") scala> val insertVals = (0 until numCols).map { _ => 1 }.mkString(",") scala> sql(s"insert into test_$numCols values ($insertVals,3)") scala> sql(s"select grouping_id(), sum(v) from test_$numCols group by grouping sets ((${cols.mkString(",")}), (${cols.init.mkString(",")}))").show(10, false) scala> sql(s"drop table test_$numCols") // numCols = 32 +-------------+------+ |grouping_id()|sum(v)| +-------------+------+ |0 |3 | |0 |3 | // Wrong Grouping ID +-------------+------+ // numCols = 31 +-------------+------+ |grouping_id()|sum(v)| +-------------+------+ |0 |3 | |1 |3 | +-------------+------+ ``` To fix this issue, this pr change code to use long values for `GROUPING_ID` instead of int values. ### Why are the changes needed? To support more cases in `GROUPING_ID`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added unit tests. Closes #26918 from maropu/FixGroupingIdIssue. Authored-by: Takeshi Yamamuro <yamamuro@apache.org> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-06 02:57:03 -05:00
- In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.1, you can set `spark.sql.ui.explainMode` to `extended`.
[SPARK-31830][SQL] Consistent error handling for datetime formatting and parsing functions ### What changes were proposed in this pull request? Currently, `date_format` and `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp`, `to_date` have different exception handling behavior for formatting datetime values. In this PR, we apply the exception handling behavior of `date_format` to `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date`. In the phase of creating the datetime formatted or formating, exceptions will be raised. e.g. ```java spark-sql> select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-aaa'); 20/05/28 15:25:38 ERROR SparkSQLDriver: Failed in [select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-aaa')] org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyyyyyy-MM-aaa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html ``` ```java spark-sql> select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-AAA'); 20/05/28 15:26:10 ERROR SparkSQLDriver: Failed in [select date_format(make_timestamp(1, 1 ,1,1,1,1), 'yyyyyyyyyyy-MM-AAA')] java.lang.IllegalArgumentException: Illegal pattern character: A ``` ```java spark-sql> select date_format(make_timestamp(1,1,1,1,1,1), 'yyyyyyyyyyy-MM-dd'); 20/05/28 15:23:23 ERROR SparkSQLDriver: Failed in [select date_format(make_timestamp(1,1,1,1,1,1), 'yyyyyyyyyyy-MM-dd')] java.lang.ArrayIndexOutOfBoundsException: 11 at java.time.format.DateTimeFormatterBuilder$NumberPrinterParser.format(DateTimeFormatterBuilder.java:2568) ``` In the phase of parsing, `DateTimeParseException | DateTimeException | ParseException` will be suppressed, but `SparkUpgradeException` will still be raised e.g. ```java spark-sql> set spark.sql.legacy.timeParserPolicy=exception; spark.sql.legacy.timeParserPolicy exception spark-sql> select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz"); 20/05/28 15:31:15 ERROR SparkSQLDriver: Failed in [select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz")] org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020-01-27T20:06:11.847-0800' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string. ``` ```java spark-sql> set spark.sql.legacy.timeParserPolicy=corrected; spark.sql.legacy.timeParserPolicy corrected spark-sql> select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz"); NULL spark-sql> set spark.sql.legacy.timeParserPolicy=legacy; spark.sql.legacy.timeParserPolicy legacy spark-sql> select to_timestamp("2020-01-27T20:06:11.847-0800", "yyyy-MM-dd'T'HH:mm:ss.SSSz"); 2020-01-28 12:06:11.847 ``` ### Why are the changes needed? Consistency ### Does this PR introduce _any_ user-facing change? Yes, invalid datetime patterns will fail `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` instead of resulting `NULL` ### How was this patch tested? add more tests Closes #28650 from yaooqinn/SPARK-31830. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-09 12:56:45 -04:00
- In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`.
[SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources ### What changes were proposed in this pull request? When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception: ``` org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: ``` ### Why are the changes needed? To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`: ```Scala org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase` ``` Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758. ### Does this PR introduce _any_ user-facing change? Yes. For the example from SPARK-32431: ORC: ```scala java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329) at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78) ... Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3 at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61) at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323) ``` JSON: ```scala +------------+ |StructColumn| +------------+ | [,,]| +------------+ ``` Parquet: ```scala +------------+ |StructColumn| +------------+ | [0,, 1]| +------------+ ``` Avro: ```scala +------------+ |StructColumn| +------------+ | [,,]| +------------+ ``` After the changes, Parquet, ORC, JSON and Avro output the same error: ```scala Found duplicate column(s) in the data schema: `camelcase`; org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67) ``` ### How was this patch tested? Run modified test suites: ``` $ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite" $ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*" ``` and added new UT to `SchemaUtilsSuite`. Closes #29234 from MaxGekk/nested-case-insensitive-column. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 02:05:55 -04:00
- In Spark 3.1, the Parquet, ORC, Avro and JSON datasources throw the exception `org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema` in read if they detect duplicate names in top-level columns as well in nested structures. The datasources take into account the SQL config `spark.sql.caseSensitive` while detecting column name duplicates.
- In Spark 3.1, structs and maps are wrapped by the `{}` brackets in casting them to strings. For instance, the `show()` action and the `CAST` expression use such brackets. In Spark 3.0 and earlier, the `[]` brackets are used for the same purpose. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`.
[SPARK-32501][SQL] Convert null to "null" in structs, maps and arrays while casting to strings ### What changes were proposed in this pull request? Convert `NULL` elements of maps, structs and arrays to the `"null"` string while converting maps/struct/array values to strings. The SQL config `spark.sql.legacy.omitNestedNullInCast.enabled` controls the behaviour. When it is `true`, `NULL` elements of structs/maps/arrays will be omitted otherwise, when it is `false`, `NULL` elements will be converted to `"null"`. ### Why are the changes needed? 1. It is impossible to distinguish empty string and null, for instance: ```scala scala> Seq(Seq(""), Seq(null)).toDF().show +-----+ |value| +-----+ | []| | []| +-----+ ``` 2. Inconsistent NULL conversions for top-level values and nested columns, for instance: ```scala scala> sql("select named_struct('c', null), null").show +---------------------+----+ |named_struct(c, NULL)|NULL| +---------------------+----+ | []|null| +---------------------+----+ ``` 3. `.show()` is different from conversions to Hive strings, and as a consequence its output is different from `spark-sql` (sql tests): ```sql spark-sql> select named_struct('c', null) as struct; {"c":null} ``` ```scala scala> sql("select named_struct('c', null) as struct").show +------+ |struct| +------+ | []| +------+ ``` 4. It is impossible to distinguish empty struct/array from struct/array with null in the current implementation: ```scala scala> Seq[Seq[String]](Seq(), Seq(null)).toDF.show() +-----+ |value| +-----+ | []| | []| +-----+ ``` ### Does this PR introduce _any_ user-facing change? Yes, before: ```scala scala> Seq(Seq(""), Seq(null)).toDF().show +-----+ |value| +-----+ | []| | []| +-----+ ``` After: ```scala scala> Seq(Seq(""), Seq(null)).toDF().show +------+ | value| +------+ | []| |[null]| +------+ ``` ### How was this patch tested? By existing test suite `CastSuite`. Closes #29311 from MaxGekk/nested-null-to-string. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-05 08:03:36 -04:00
- In Spark 3.1, NULL elements of structures, arrays and maps are converted to "null" in casting them to strings. In Spark 3.0 or earlier, NULL elements are converted to empty strings. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.castComplexTypesToString.enabled` to `true`.
- In Spark 3.1, when `spark.sql.ansi.enabled` is false, Spark always returns null if the sum of decimal type column overflows. In Spark 3.0 or earlier, in the case, the sum of decimal type column may return null or incorrect result, or even fails at runtime (depending on the actual query plan execution).
[SPARK-32516][SQL][FOLLOWUP] 'path' option cannot coexist with path parameter for DataFrameWriter.save(), DataStreamReader.load() and DataStreamWriter.start() ### What changes were proposed in this pull request? This is a follow up PR to #29328 to apply the same constraint where `path` option cannot coexist with path parameter to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`. ### Why are the changes needed? The current behavior silently overwrites the `path` option if path parameter is passed to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`. For example, ``` Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2") ``` will write the result to `/tmp/path2`. ### Does this PR introduce _any_ user-facing change? Yes, if `path` option coexists with path parameter to any of the above methods, it will throw `AnalysisException`: ``` scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2") org.apache.spark.sql.AnalysisException: There is a 'path' option set and save() is called with a path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set 'spark.sql.legacy.pathOptionBehavior.enabled' to 'true'.; ``` The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`. ### How was this patch tested? Added new tests. Closes #29543 from imback82/path_option. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 02:21:04 -04:00
- In Spark 3.1, `path` option cannot coexist when the following methods are called with path parameter(s): `DataFrameReader.load()`, `DataFrameWriter.save()`, `DataStreamReader.load()`, or `DataStreamWriter.start()`. In addition, `paths` option cannot coexist for `DataFrameReader.load()`. For example, `spark.read.format("csv").option("path", "/tmp").load("/tmp2")` or `spark.read.option("path", "/tmp").csv("/tmp2")` will throw `org.apache.spark.sql.AnalysisException`. In Spark version 3.0 and below, `path` option is overwritten if one path parameter is passed to above methods; `path` option is added to the overall paths if multiple path parameters are passed to `DataFrameReader.load()`. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.
[SPARK-32516][SQL] 'path' option cannot coexist with load()'s path parameters ### What changes were proposed in this pull request? This PR proposes to make the behavior consistent for the `path` option when loading dataframes with a single path (e.g, `option("path", path).format("parquet").load(path)` vs. `option("path", path).parquet(path)`) by disallowing `path` option to coexist with `load`'s path parameters. ### Why are the changes needed? The current behavior is inconsistent: ```scala scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test") scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show +-----+ |value| +-----+ | 1| +-----+ scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show +-----+ |value| +-----+ | 1| | 1| +-----+ ``` ### Does this PR introduce _any_ user-facing change? Yes, now if the `path` option is specified along with `load`'s path parameters, it would fail: ```scala scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test") scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.; at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:232) ... 47 elided scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.; at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:250) at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:778) at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:756) ... 47 elided ``` The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`. ### How was this patch tested? Added a test Closes #29328 from imback82/dfw_option. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-24 12:30:30 -04:00
- In Spark 3.1, `IllegalArgumentException` is returned for the incomplete interval literals, e.g. `INTERVAL '1'`, `INTERVAL '1 DAY 2'`, which are invalid. In Spark 3.0, these literals result in `NULL`s.
[SPARK-20202][BUILD][SQL] Remove references to org.spark-project.hive (Hive 1.2.1) ### What changes were proposed in this pull request? As of today, - SPARK-30034 Apache Spark 3.0.0 switched its default Hive execution engine from Hive 1.2 to Hive 2.3. This removes the direct dependency to the forked Hive 1.2.1 in maven repository. - SPARK-32981 Apache Spark 3.1.0(`master` branch) removed Hive 1.2 related artifacts from Apache Spark binary distributions. This PR(SPARK-20202) aims to remove the following usage of unofficial Apache Hive fork completely from Apache Spark master for Apache Spark 3.1.0. ``` <hive.group>org.spark-project.hive</hive.group> <hive.version>1.2.1.spark2</hive.version> ``` For the forked Hive 1.2.1.spark2 users, Apache Spark 2.4(LTS) and 3.0 (~ 2021.12) will provide it. ### Why are the changes needed? - First, Apache Spark community should not use the unofficial forked release of another Apache project. - Second, Apache Hive 1.2.1 was released at 2015-06-26 and the forked Hive `1.2.1.spark2` exposed many unfixable bugs in Apache because the forked `1.2.1.spark2` is not maintained at all. Apache Hive 2.3.0 was released at 2017-07-19 and it has been used with less number of bugs compared with `1.2.1.spark2`. Many bugs still exist in `hive-1.2` profile and new Apache Spark unit tests are added with `HiveUtils.isHive23` condition so far. ### Does this PR introduce _any_ user-facing change? No. This is a dev-only change. PRBuilder will not accept `[test-hive1.2]` on master and `branch-3.1`. ### How was this patch tested? 1. SBT/Hadoop 3.2/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129366) 2. SBT/Hadoop 2.7/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129382) 3. SBT/Hadoop 3.2/Hive 1.2 (This has not been supported already due to Hive 1.2 doesn't work with Hadoop 3.2.) 4. SBT/Hadoop 2.7/Hive 1.2 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129383, This is rejected) Closes #29936 from dongjoon-hyun/SPARK-REMOVE-HIVE1. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-05 18:29:56 -04:00
- In Spark 3.1, we remove the built-in Hive 1.2. You need to migrate your custom SerDes to Hive 2.3. See [HIVE-15167](https://issues.apache.org/jira/browse/HIVE-15167) for more details.
- In Spark 3.1, loading and saving of timestamps from/to parquet files fails if the timestamps are before 1900-01-01 00:00:00Z, and loaded (saved) as the INT96 type. In Spark 3.0, the actions don't fail but might lead to shifting of the input timestamps due to rebasing from/to Julian to/from Proleptic Gregorian calendar. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.parquet.int96RebaseModeInRead` or/and `spark.sql.legacy.parquet.int96RebaseModeInWrite` to `LEGACY`.
[SPARK-33270][SQL] Return SQL schema instead of Catalog string from the `SchemaOfJson` expression ### What changes were proposed in this pull request? Return schema in SQL format instead of Catalog string from the `SchemaOfJson` expression. ### Why are the changes needed? In some cases, `from_json()` cannot parse schemas returned by `schema_of_json`, for instance, when JSON fields have spaces (gaps). Such fields will be quoted after the changes, and can be parsed by `from_json()`. Here is the example: ```scala val in = Seq("""{"a b": 1}""").toDS() in.select(from_json('value, schema_of_json("""{"a b": 100}""")) as "parsed") ``` raises the exception: ``` == SQL == struct<a b:bigint> ------^^^ 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.catalyst.parser.AbstractSqlParser.parseTableSchema(ParseDriver.scala:76) at org.apache.spark.sql.types.DataType$.fromDDL(DataType.scala:131) at org.apache.spark.sql.catalyst.expressions.ExprUtils$.evalTypeExpr(ExprUtils.scala:33) at org.apache.spark.sql.catalyst.expressions.JsonToStructs.<init>(jsonExpressions.scala:537) at org.apache.spark.sql.functions$.from_json(functions.scala:4141) ``` ### Does this PR introduce _any_ user-facing change? Yes. For example, `schema_of_json` for the input `{"col":0}`. Before: `struct<col:bigint>` After: `STRUCT<`col`: BIGINT>` ### How was this patch tested? By existing test suites `JsonFunctionsSuite` and `JsonExpressionsSuite`. Closes #30172 from MaxGekk/schema_of_json-sql-schema. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-28 21:30:41 -04:00
- In Spark 3.1, the `schema_of_json` and `schema_of_csv` functions return the schema in the SQL format in which field names are quoted. In Spark 3.0, the function returns a catalog string without field quoting and in lower case.
[SPARK-20202][BUILD][SQL] Remove references to org.spark-project.hive (Hive 1.2.1) ### What changes were proposed in this pull request? As of today, - SPARK-30034 Apache Spark 3.0.0 switched its default Hive execution engine from Hive 1.2 to Hive 2.3. This removes the direct dependency to the forked Hive 1.2.1 in maven repository. - SPARK-32981 Apache Spark 3.1.0(`master` branch) removed Hive 1.2 related artifacts from Apache Spark binary distributions. This PR(SPARK-20202) aims to remove the following usage of unofficial Apache Hive fork completely from Apache Spark master for Apache Spark 3.1.0. ``` <hive.group>org.spark-project.hive</hive.group> <hive.version>1.2.1.spark2</hive.version> ``` For the forked Hive 1.2.1.spark2 users, Apache Spark 2.4(LTS) and 3.0 (~ 2021.12) will provide it. ### Why are the changes needed? - First, Apache Spark community should not use the unofficial forked release of another Apache project. - Second, Apache Hive 1.2.1 was released at 2015-06-26 and the forked Hive `1.2.1.spark2` exposed many unfixable bugs in Apache because the forked `1.2.1.spark2` is not maintained at all. Apache Hive 2.3.0 was released at 2017-07-19 and it has been used with less number of bugs compared with `1.2.1.spark2`. Many bugs still exist in `hive-1.2` profile and new Apache Spark unit tests are added with `HiveUtils.isHive23` condition so far. ### Does this PR introduce _any_ user-facing change? No. This is a dev-only change. PRBuilder will not accept `[test-hive1.2]` on master and `branch-3.1`. ### How was this patch tested? 1. SBT/Hadoop 3.2/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129366) 2. SBT/Hadoop 2.7/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129382) 3. SBT/Hadoop 3.2/Hive 1.2 (This has not been supported already due to Hive 1.2 doesn't work with Hadoop 3.2.) 4. SBT/Hadoop 2.7/Hive 1.2 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129383, This is rejected) Closes #29936 from dongjoon-hyun/SPARK-REMOVE-HIVE1. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-05 18:29:56 -04:00
- In Spark 3.1, refreshing a table will trigger an uncache operation for all other caches that reference the table, even if the table itself is not cached. In Spark 3.0 the operation will only be triggered if the table itself is cached.
- In Spark 3.1, creating or altering a permanent view will capture runtime SQL configs and store them as view properties. These configs will be applied during the parsing and analysis phases of the view resolution. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.useCurrentConfigsForView` to `true`.
- In Spark 3.1, the temporary view will have same behaviors with the permanent view, i.e. capture and store runtime SQL configs, SQL text, catalog and namespace. The capatured view properties will be applied during the parsing and analysis phases of the view resolution. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.storeAnalyzedPlanForView` to `true`.
- In Spark 3.1, temporary view created via `CACHE TABLE ... AS SELECT` will also have the same behavior with permanent view. In particular, when the temporary view is dropped, Spark will invalidate all its cache dependents, as well as the cache for the temporary view itself. This is different from Spark 3.0 and below, which only does the latter. To restore the previous behavior, you can set `spark.sql.legacy.storeAnalyzedPlanForView` to `true`.
- Since Spark 3.1, CHAR/CHARACTER and VARCHAR types are supported in the table schema. Table scan/insertion will respect the char/varchar semantic. If char/varchar is used in places other than table schema, an exception will be thrown (CAST is an exception that simply treats char/varchar as string like before). To restore the behavior before Spark 3.1, which treats them as STRING types and ignores a length parameter, e.g. `CHAR(4)`, you can set `spark.sql.legacy.charVarcharAsString` to `true`.
- In Spark 3.1, `AnalysisException` is replaced by its sub-classes that are thrown for tables from Hive external catalog in the following situations:
* `ALTER TABLE .. ADD PARTITION` throws `PartitionsAlreadyExistException` if new partition exists already
* `ALTER TABLE .. DROP PARTITION` throws `NoSuchPartitionsException` for not existing partitions
## Upgrading from Spark SQL 3.0.1 to 3.0.2
- In Spark 3.0.2, `AnalysisException` is replaced by its sub-classes that are thrown for tables from Hive external catalog in the following situations:
* `ALTER TABLE .. ADD PARTITION` throws `PartitionsAlreadyExistException` if new partition exists already
* `ALTER TABLE .. DROP PARTITION` throws `NoSuchPartitionsException` for not existing partitions
- In Spark 3.0.2, `PARTITION(col=null)` is always parsed as a null literal in the partition spec. In Spark 3.0.1 or earlier, it is parsed as a string literal of its text representation, e.g., string "null", if the partition column is string type. To restore the legacy behavior, you can set `spark.sql.legacy.parseNullPartitionSpecAsStringLiteral` as true.
- In Spark 3.0.0, the output schema of `SHOW DATABASES` becomes `namespace: string`. In Spark version 2.4 and earlier, the schema was `databaseName: string`. Since Spark 3.0.2, you can restore the old schema by setting `spark.sql.legacy.keepCommandOutputSchema` to `true`.
## Upgrading from Spark SQL 3.0 to 3.0.1
- In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference.
[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 3.0
### Dataset/DataFrame APIs
[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 Dataset and DataFrame API `unionAll` is no longer deprecated. It is an alias for `union`.
[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 2.4 and below, `Dataset.groupByKey` results to a grouped dataset with key attribute is wrongly named as "value", if the key is non-struct type, for example, int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries unexpected. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behavior is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`.
[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 column metadata will always be propagated in the API `Column.name` and `Column.as`. In Spark version 2.4 and earlier, the metadata of `NamedExpression` is set as the `explicitMetadata` for the new column at the time the API is called, it won't change even if the underlying `NamedExpression` changes metadata. To restore the behavior before Spark 2.4, you can use the API `as(alias: String, metadata: Metadata)` with explicit metadata.
### DDL Statements
[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, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception is thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and below, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to an 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
- The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set.
[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
- Spark 2.4 and below: the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. In 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`.
- Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and below, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. In Spark 3.0, cache name and storage level are first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing.
- In Spark 3.0, the properties listing below become reserved; commands fail if you specify reserved properties in places like `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER TABLE ... SET TBLPROPERTIES`. You need their specific clauses to specify them, for example, `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. You can set `spark.sql.legacy.notReserveProperties` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, for example: `SET DBPROPERTIES('location'='/tmp')` will have no effect. In Spark version 2.4 and below, these properties are neither reserved nor have side effects, for example, `SET DBPROPERTIES('location'='/tmp')` do not change the location of the database but only create a headless property just like `'a'='b'`.
| Property (case sensitive) | Database Reserved | Table Reserved | Remarks |
| ------------------------- | ----------------- | -------------- | ------- |
| provider | no | yes | For tables, use the `USING` clause to specify it. Once set, it can't be changed. |
| location | yes | yes | For databases and tables, use the `LOCATION` clause to specify it. |
| owner | yes | yes | For databases and tables, it is determined by the user who runs spark and create the table. |
- In Spark 3.0, you can use `ADD FILE` to add file directories as well. Earlier you could add only single files using this command. To restore the behavior of earlier versions, set `spark.sql.legacy.addSingleFileInAddFile` to `true`.
- In Spark 3.0, `SHOW TBLPROPERTIES` throws `AnalysisException` if the table does not exist. In Spark version 2.4 and below, this scenario caused `NoSuchTableException`.
- In Spark 3.0, `SHOW CREATE TABLE table_identifier` always returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, use `SHOW CREATE TABLE table_identifier AS SERDE` command instead.
- In Spark 3.0, column of CHAR type is not allowed in non-Hive-Serde tables, and CREATE/ALTER TABLE commands will fail if CHAR type is detected. Please use STRING type instead. In Spark version 2.4 and below, CHAR type is treated as STRING type and the length parameter is simply ignored.
### UDFs and Built-in Functions
[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 `date_add` and `date_sub` functions accepts only int, smallint, tinyint as the 2nd argument; fractional and non-literal strings are not valid anymore, for example: `date_add(cast('1964-05-23' as date), '12.34')` causes `AnalysisException`. Note that, string literals are still allowed, but Spark will throw `AnalysisException` if the string content is not a valid integer. In Spark version 2.4 and below, if the 2nd argument is fractional or string value, it is coerced to int value, and the result is a date value of `1964-06-04`.
[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 function `percentile_approx` and its alias `approx_percentile` only accept integral value with range in `[1, 2147483647]` as its 3rd argument `accuracy`, fractional and string types are disallowed, for example, `percentile_approx(10.0, 0.2, 1.8D)` causes `AnalysisException`. In Spark version 2.4 and below, if `accuracy` is fractional or string value, it is coerced to an int value, `percentile_approx(10.0, 0.2, 1.8D)` is operated as `percentile_approx(10.0, 0.2, 1)` which results in `10.0`.
[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, an analysis exception is thrown when hash expressions are applied on elements of `MapType`. To restore the behavior before Spark 3.0, set `spark.sql.legacy.allowHashOnMapType` to `true`.
[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, when the `array`/`map` function is called without any parameters, it returns an empty collection with `NullType` as element type. In Spark version 2.4 and below, it returns an empty collection with `StringType` as element type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createEmptyCollectionUsingStringType` to `true`.
[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 `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)`.
[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 below, you can create map values with map type key via built-in function such as `CreateMap`, `MapFromArrays`, etc. In Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can use `map_entries` function to convert map to array<struct<key, value>> as a workaround. In addition, users can still read map values with map type key from data source or Java/Scala collections, though it is discouraged.
[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 below, you can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, for example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. In Spark 3.0, Spark throws `RuntimeException` when duplicated keys are found. You can set `spark.sql.mapKeyDedupPolicy` to `LAST_WIN` to deduplicate map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (for example, Parquet), the behavior is undefined.
[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, using `org.apache.spark.sql.functions.udf(AnyRef, DataType)` is not allowed by default. Remove the return type parameter to automatically switch to typed Scala udf is recommended, or set `spark.sql.legacy.allowUntypedScalaUDF` to true to keep using it. In Spark version 2.4 and below, if `org.apache.spark.sql.functions.udf(AnyRef, DataType)` gets a Scala closure with primitive-type argument, the returned UDF returns null if the input values is null. However, in Spark 3.0, the UDF returns the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` returns null in Spark 2.4 and below if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default.
[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, a higher-order function `exists` follows the three-valued boolean logic, that is, if the `predicate` returns any `null`s and no `true` is obtained, then `exists` returns `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` is `null`. The previous behavior can be restored by setting `spark.sql.legacy.followThreeValuedLogicInArrayExists` 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
- In Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and below, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`.
[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 below, the `current_timestamp` function returns a timestamp with millisecond resolution only. In Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution.
- In Spark 3.0, a 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and below, the 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases.
[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
- 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()`.
- In Spark 3.0, the `cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in a case-insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behavior change is illustrated in the table below:
| Operation | Result before Spark 3.0 | Result in Spark 3.0 |
| --------- | ----------------------- | ------------------- |
| CAST('infinity' AS DOUBLE) | NULL | Double.PositiveInfinity |
| CAST('+infinity' AS DOUBLE) | NULL | Double.PositiveInfinity |
| CAST('inf' AS DOUBLE) | NULL | Double.PositiveInfinity |
| CAST('inf' AS DOUBLE) | NULL | Double.PositiveInfinity |
| CAST('-infinity' AS DOUBLE) | NULL | Double.NegativeInfinity |
| CAST('-inf' AS DOUBLE) | NULL | Double.NegativeInfinity |
| CAST('infinity' AS FLOAT) | NULL | Float.PositiveInfinity |
| CAST('+infinity' AS FLOAT) | NULL | Float.PositiveInfinity |
| CAST('inf' AS FLOAT) | NULL | Float.PositiveInfinity |
| CAST('+inf' AS FLOAT) | NULL | Float.PositiveInfinity |
| CAST('-infinity' AS FLOAT) | NULL | Float.NegativeInfinity |
| CAST('-inf' AS FLOAT) | NULL | Float.NegativeInfinity |
| CAST('nan' AS DOUBLE) | NULL | Double.Nan |
| CAST('nan' AS FLOAT) | NULL | Float.NaN |
- In Spark 3.0, when casting interval values to string type, there is no "interval" prefix, for example, `1 days 2 hours`. In Spark version 2.4 and below, the string contains the "interval" prefix like `interval 1 days 2 hours`.
- In Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces (<= ASCII 32) will be trimmed before converted to these type values, for example, `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and below, when casting string to integrals and booleans, it does not trim the whitespaces from both ends; the foregoing results is `null`, while to datetimes, only the trailing spaces (= ASCII 32) are removed.
### Query Engine
- In Spark version 2.4 and below, 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. These queries are treated as invalid in Spark 3.0.
- In Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception.
- In Spark 3.0, numbers written in scientific notation(for example, `1E2`) would be parsed as Double. In Spark version 2.4 and below, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`.
- In Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`.
- In Spark 3.0, negative scale of decimal is not allowed by default, for example, data type of literal like `1E10BD` is `DecimalType(11, 0)`. In Spark version 2.4 and below, it was `DecimalType(2, -9)`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.allowNegativeScaleOfDecimal` to `true`.
- In Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with an integral string representation is coerced to a double value, for example, `+'1'` returns `1.0`. In Spark version 2.4 and below, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, for example, `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, for example, in Spark 2.4, the result of `+'1'` is string `1`.
- In Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`.
- In Spark 3.0, `spark.sql.legacy.ctePrecedencePolicy` is introduced to control the behavior for name conflicting in the nested WITH clause. By default value `EXCEPTION`, Spark throws an AnalysisException, it forces users to choose the specific substitution order they wanted. If set to `CORRECTED` (which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config to `false`, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `2`, while setting it to `LEGACY`, the result is `1` which is the behavior in version 2.4 and below.
- In Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join.
- In Spark version 2.4 and below, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys, and join keys. In Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and below.
- In Spark version 2.4 and below, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. In Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`.
- In Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and so on. Spark 3.0 uses Java 8 API classes from the `java.time` packages that are based on [ISO chronology](https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and below, those operations are performed using the hybrid calendar ([Julian + Gregorian](https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API:
* Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in [Datetime Patterns for Formatting and Parsing](sql-ref-datetime-pattern.html),
which is implemented via [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [SimpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`.
* The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone.
* The JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and below, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone.
* Formatting `TIMESTAMP` and `DATE` literals.
* Creating typed `TIMESTAMP` and `DATE` literals from strings. In Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. When the input string does not contain information about time zone, the time zone from the SQL config `spark.sql.session.timeZone` is used in that case. In Spark version 2.4 and below, the conversion is based on JVM system time zone. The different sources of the default time zone may change the behavior of typed `TIMESTAMP` and `DATE` literals.
- In Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and below, the conversion uses the default time zone of the Java virtual machine.
- In Spark 3.0, Spark casts `String` to `Date/Timestamp` in binary comparisons with dates/timestamps. The previous behavior of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString.enabled` to `true`.
- In Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that are converted to ordinary date or timestamp values when read. The following string values are supported for dates:
* `epoch [zoneId]` - 1970-01-01
* `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone`
* `yesterday [zoneId]` - the current date - 1
* `tomorrow [zoneId]` - the current date + 1
* `now` - the date of running the current query. It has the same notion as today
For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values:
* `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero)
* `today [zoneId]` - midnight today
* `yesterday [zoneId]` - midnight yesterday
* `tomorrow [zoneId]` - midnight tomorrow
* `now` - current query start time
For example `SELECT timestamp 'tomorrow';`.
- Since Spark 3.0, when using `EXTRACT` expression to extract the second field from date/timestamp values, the result will be a `DecimalType(8, 6)` value with 2 digits for second part, and 6 digits for the fractional part with microsecond precision. e.g. `extract(second from to_timestamp('2019-09-20 10:10:10.1'))` results `10.100000`. In Spark version 2.4 and earlier, it returns an `IntegerType` value and the result for the former example is `10`.
- In Spark 3.0, datetime pattern letter `F` is **aligned day of week in month** that represents the concept of the count of days within the period of a week where the weeks are aligned to the start of the month. In Spark version 2.4 and earlier, it is **week of month** that represents the concept of the count of weeks within the month where weeks start on a fixed day-of-week, e.g. `2020-07-30` is 30 days (4 weeks and 2 days) after the first day of the month, so `date_format(date '2020-07-30', 'F')` returns 2 in Spark 3.0, but as a week count in Spark 2.x, it returns 5 because it locates in the 5th week of July 2020, where week one is 2020-07-01 to 07-04.
### Data Sources
- In Spark version 2.4 and below, when reading a Hive SerDe table with Spark native data sources(parquet/orc), Spark infers the actual file schema and update the table schema in metastore. In Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`.
- In Spark version 2.4 and below, partition column value is converted as null if it can't be casted to corresponding user provided schema. In 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`.
- In Spark 3.0, if files or subdirectories disappear during recursive directory listing (that is, they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time.
[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
- In Spark version 2.4 and below, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType`, `DoubleType`, `DateType` and `TimestampType`, it fails on empty strings and throws exceptions. Spark 3.0 disallows empty strings and will throw an exception for data types except for `StringType` and `BinaryType`. The previous behavior of allowing an empty string can be restored by setting `spark.sql.legacy.json.allowEmptyString.enabled` to `true`.
- In Spark version 2.4 and below, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. In Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully.
- In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inference.
- In Spark version 2.4 and below, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. In Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully.
- In Spark 3.0, when Avro files are written with user provided schema, the fields are matched by field names between catalyst schema and Avro schema instead of positions.
- In Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark throws runtime NullPointerException if any of the records contains null.
### Others
- In Spark version 2.4, when a Spark session is created via `cloneSession()`, the newly created Spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent Spark session. In Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. You can restore the old behavior by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`.
- In Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it falls back to the `hive-site.xml` file present in the `Hadoop configuration` of `SparkContext`.
- In Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example:
| Query | Spark 2.4 | Spark 3.0 |
| ----- | --------- | --------- |
|`SELECT CAST(1 AS decimal(38, 18));` | 1 | 1.000000000000000000 |
- In Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3 and it brings following impacts:
* You may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of the Hive metastore you want to connect to. For example: set `spark.sql.hive.metastore.version` to `1.2.1` and `spark.sql.hive.metastore.jars` to `maven` if your Hive metastore version is 1.2.1.
* You need to migrate your custom SerDes to Hive 2.3 or build your own Spark with `hive-1.2` profile. See [HIVE-15167](https://issues.apache.org/jira/browse/HIVE-15167) for more details.
* The decimal string representation can be different between Hive 1.2 and Hive 2.3 when using `TRANSFORM` operator in SQL for script transformation, which depends on hive's behavior. In Hive 1.2, the string representation omits trailing zeroes. But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary.
[SPARK-27686][DOC][SQL] Update migration guide for make Hive 2.3 dependency by default ### What changes were proposed in this pull request? We have upgraded the built-in Hive from 1.2 to 2.3. This may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of your Hive metastore. Example: ``` --conf spark.sql.hive.metastore.version=1.2.1 --conf spark.sql.hive.metastore.jars=/root/hive-1.2.1-lib/* ``` Otherwise: ``` org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: Unable to fetch table spark_27686. Invalid method name: 'get_table_req'; at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:110) at org.apache.spark.sql.hive.HiveExternalCatalog.tableExists(HiveExternalCatalog.scala:841) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.tableExists(ExternalCatalogWithListener.scala:146) at org.apache.spark.sql.catalyst.catalog.SessionCatalog.tableExists(SessionCatalog.scala:431) at org.apache.spark.sql.execution.command.CreateDataSourceTableCommand.run(createDataSourceTables.scala:52) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68) at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79) at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:226) at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3487) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3485) at org.apache.spark.sql.Dataset.<init>(Dataset.scala:226) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607) ... 47 elided Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Unable to fetch table spark_27686. Invalid method name: 'get_table_req' at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:1282) at org.apache.spark.sql.hive.client.HiveClientImpl.getRawTableOption(HiveClientImpl.scala:422) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$tableExists$1(HiveClientImpl.scala:436) at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:322) at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:256) at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:255) at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:305) at org.apache.spark.sql.hive.client.HiveClientImpl.tableExists(HiveClientImpl.scala:436) at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$tableExists$1(HiveExternalCatalog.scala:841) at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:100) ... 63 more Caused by: org.apache.thrift.TApplicationException: Invalid method name: 'get_table_req' at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_table_req(ThriftHiveMetastore.java:1567) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_table_req(ThriftHiveMetastore.java:1554) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getTable(HiveMetaStoreClient.java:1350) at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.getTable(SessionHiveMetaStoreClient.java:127) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:173) at com.sun.proxy.$Proxy38.getTable(Unknown Source) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:2336) at com.sun.proxy.$Proxy38.getTable(Unknown Source) at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:1274) ... 74 more ``` ### Why are the changes needed? Improve documentation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? ```SKIP_API=1 jekyll build```: ![image](https://user-images.githubusercontent.com/5399861/73531432-67a50b80-4455-11ea-9401-5cad12fd3d14.png) Closes #27161 from wangyum/SPARK-27686. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-01 23:50:47 -05:00
## Upgrading from Spark SQL 2.4.7 to 2.4.8
- In Spark 2.4.8, `AnalysisException` is replaced by its sub-classes that are thrown for tables from Hive external catalog in the following situations:
* `ALTER TABLE .. ADD PARTITION` throws `PartitionsAlreadyExistException` if new partition exists already
* `ALTER TABLE .. DROP PARTITION` throws `NoSuchPartitionsException` for not existing partitions
## Upgrading from Spark SQL 2.4.5 to 2.4.6
- In Spark 2.4.6, the `RESET` command does not reset the static SQL configuration values to the default. It only clears the runtime SQL configuration values.
## 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. For example, 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.
[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 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, i.e. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive modulus (`pmod`).
[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
- 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, i.e. 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.
[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
- 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/org/apache/spark/sql/SQLContext.html#read:DataFrameReader">Scala</a>,
[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
<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/org/apache/spark/sql/DataFrame.html#write:DataFrameWriter">Scala</a>,
[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
<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.8 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)).
[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
#### 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`
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
* `DISTRIBUTE BY`
[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
* `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)
[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
* 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`
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
* `LEFT ANTI JOIN`
[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
* `CROSS JOIN`
* Unions
* Sub-queries
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
* Sub-queries in the FROM Clause
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
```SELECT col FROM (SELECT a + b AS col FROM t1) t2```
* Sub-queries in WHERE Clause
* Correlated or non-correlated IN and NOT IN statement in WHERE Clause
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
```
SELECT col FROM t1 WHERE col IN (SELECT a FROM t2 WHERE t1.a = t2.a)
SELECT col FROM t1 WHERE col IN (SELECT a FROM t2)
```
* Correlated or non-correlated EXISTS and NOT EXISTS statement in WHERE Clause
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
```
SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t1.a = t2.a AND t2.a > 10)
SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t2.a > 10)
```
* Non-correlated IN and NOT IN statement in JOIN Condition
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND t1.a IN (SELECT a FROM t3)```
[SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu <angers.zhu@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 23:55:29 -05:00
* Non-correlated EXISTS and NOT EXISTS statement in JOIN Condition
```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND EXISTS (SELECT * FROM t3 WHERE t3.a > 10)```
[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
* 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.
* `CAST(n AS TIMESTAMP)` If n is integral numbers, Hive treats n as milliseconds, Spark SQL treats n as seconds.