Commit graph

4741 commits

Author SHA1 Message Date
Liang-Chi Hsieh 056b62264b [SPARK-33263][SS] Configurable StateStore compression codec
### What changes were proposed in this pull request?

This patch proposes to make StateStore compression codec configurable.

### Why are the changes needed?

Currently the compression codec of StateStore is not configurable and hard-coded to be lz4. It is better if we can follow Spark other modules to configure the compression codec of StateStore. For example, we can choose zstd codec and zstd is configurable with different compression level.

### Does this PR introduce _any_ user-facing change?

Yes, after this change users can config different codec for StateStore.

### How was this patch tested?

Unit test.

Closes #30162 from viirya/SPARK-33263.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-29 07:44:44 -07:00
Max Gekk b409025641 [SPARK-33281][SQL] Return SQL schema instead of Catalog string from the SchemaOfCsv expression
### What changes were proposed in this pull request?
Return schema in SQL format instead of Catalog string from the SchemaOfCsv expression.

### Why are the changes needed?
To unify output of the `schema_of_json()` and `schema_of_csv()`.

### Does this PR introduce _any_ user-facing change?
Yes, they can but `schema_of_csv()` is usually used in combination with `from_csv()`, so, the format of schema shouldn't be much matter.

Before:
```
> SELECT schema_of_csv('1,abc');
  struct<_c0:int,_c1:string>
```

After:
```
> SELECT schema_of_csv('1,abc');
  STRUCT<`_c0`: INT, `_c1`: STRING>
```

### How was this patch tested?
By existing test suites `CsvFunctionsSuite` and `CsvExpressionsSuite`.

Closes #30180 from MaxGekk/schema_of_csv-sql-schema.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-29 21:02:10 +09:00
Max Gekk 9d5e48ea95 [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-29 10:30:41 +09:00
Nathan Wreggit c592ae6ed8 [SQL][MINOR] Update from_unixtime doc
### What changes were proposed in this pull request?
This PR fixes from_unixtime documentation to show that fmt is optional parameter.

### Does this PR introduce _any_ user-facing change?
Yes, documentation update.
**Before change:**
![image](https://user-images.githubusercontent.com/4176173/97497659-18c6cc80-1928-11eb-93d8-453ef627ac7c.png)

**After change:**
![image](https://user-images.githubusercontent.com/4176173/97496153-c5537f00-1925-11eb-8102-457e85e019d5.png)

### How was this patch tested?
Style check using: ./dev/run-tests
Manual check and screenshotting with: ./sql/create-docs.sh
Manual verification of behavior with latest spark-sql binary.

Closes #30176 from Obbay2/from_unixtime_doc.

Authored-by: Nathan Wreggit <obbay2@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-29 10:28:50 +09:00
Wenchen Fan 2639ad43cb [SPARK-33272][SQL] prune the attributes mapping in QueryPlan.transformUpWithNewOutput
### What changes were proposed in this pull request?

For complex query plans, `QueryPlan.transformUpWithNewOutput` will keep accumulating the attributes mapping to be propagated, which may hurt performance. This PR prunes the attributes mapping before propagating.

### Why are the changes needed?

A simple perf improvement.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

existing tests

Closes #30173 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-29 07:37:16 +09:00
Jungtaek Lim (HeartSaVioR) a744fea3be [SPARK-33267][SQL] Fix NPE issue on 'In' filter when one of values contains null
### What changes were proposed in this pull request?

This PR proposes to fix the NPE issue on `In` filter when one of values contain null. In real case, you can trigger this issue when you try to push down the filter with `in (..., null)` against V2 source table. `DataSourceStrategy` caches the mapping (filter instance -> expression) in HashMap, which leverages hash code on the key, hence it could trigger the NPE issue.

### Why are the changes needed?

This is an obvious bug as `In` filter doesn't care about null value when calculating hash code.

### Does this PR introduce _any_ user-facing change?

Yes, previously the query with having `null` in "in" condition against data source V2 source table supporting push down filter failed with NPE, whereas after the PR the query will not fail.

### How was this patch tested?

UT added. The new UT fails without the PR and passes with the PR.

Closes #30170 from HeartSaVioR/SPARK-33267.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-28 10:00:29 -07:00
Takeshi Yamamuro a6216e2446 [SPARK-33268][SQL][PYTHON] Fix bugs for casting data from/to PythonUserDefinedType
### What changes were proposed in this pull request?

This PR intends to fix bus for casting data from/to PythonUserDefinedType. A sequence of queries to reproduce this issue is as follows;
```
>>> from pyspark.sql import Row
>>> from pyspark.sql.functions import col
>>> from pyspark.sql.types import *
>>> from pyspark.testing.sqlutils import *
>>>
>>> row = Row(point=ExamplePoint(1.0, 2.0))
>>> df = spark.createDataFrame([row])
>>> df.select(col("point").cast(PythonOnlyUDT()))
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/maropu/Repositories/spark/spark-master/python/pyspark/sql/dataframe.py", line 1402, in select
    jdf = self._jdf.select(self._jcols(*cols))
  File "/Users/maropu/Repositories/spark/spark-master/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/Users/maropu/Repositories/spark/spark-master/python/pyspark/sql/utils.py", line 111, in deco
    return f(*a, **kw)
  File "/Users/maropu/Repositories/spark/spark-master/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 328, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o44.select.
: java.lang.NullPointerException
	at org.apache.spark.sql.types.UserDefinedType.acceptsType(UserDefinedType.scala:84)
	at org.apache.spark.sql.catalyst.expressions.Cast$.canCast(Cast.scala:96)
	at org.apache.spark.sql.catalyst.expressions.CastBase.checkInputDataTypes(Cast.scala:267)
	at org.apache.spark.sql.catalyst.expressions.CastBase.resolved$lzycompute(Cast.scala:290)
	at org.apache.spark.sql.catalyst.expressions.CastBase.resolved(Cast.scala:290)
```
A root cause of this issue is that, since `PythonUserDefinedType#userClassis` always null, `isAssignableFrom` in `UserDefinedType#acceptsType` throws a null exception. To fix it, this PR defines  `acceptsType` in `PythonUserDefinedType` and filters out the null case in `UserDefinedType#acceptsType`.

### Why are the changes needed?

Bug fixes.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added tests.

Closes #30169 from maropu/FixPythonUDTCast.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-28 08:33:02 -07:00
gengjiaan 3c3ad5f7c0 [SPARK-32934][SQL] Improve the performance for NTH_VALUE and reactor the OffsetWindowFunction
### What changes were proposed in this pull request?
Spark SQL supports some window function like `NTH_VALUE`.
If we specify window frame like `UNBOUNDED PRECEDING AND CURRENT ROW` or `UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`, we can elimate some calculations.
For example: if we execute the SQL show below:
```
SELECT NTH_VALUE(col,
         2) OVER(ORDER BY rank UNBOUNDED PRECEDING
        AND CURRENT ROW)
FROM tab;
```
The output for row number greater than 1, return the fixed value. otherwise, return null. So we just calculate the value once and notice whether the row number less than 2.
`UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING` is simpler.

### Why are the changes needed?
Improve the performance for `NTH_VALUE`, `FIRST_VALUE` and `LAST_VALUE`.

### Does this PR introduce _any_ user-facing change?
 'No'.

### How was this patch tested?
Jenkins test.

Closes #29800 from beliefer/optimize-nth_value.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-28 06:40:23 +00:00
allisonwang-db 9fb45361fd [SPARK-33183][SQL] Fix Optimizer rule EliminateSorts and add a physical rule to remove redundant sorts
### What changes were proposed in this pull request?
This PR aims to fix a correctness bug in the optimizer rule `EliminateSorts`. It also adds a new physical rule to remove redundant sorts that cannot be eliminated in the Optimizer rule after the bugfix.

### Why are the changes needed?
A global sort should not be eliminated even if its child is ordered since we don't know if its child ordering is global or local. For example, in the following scenario, the first sort shouldn't be removed because it has a stronger guarantee than the second sort even if the sort orders are the same for both sorts.

```
Sort(orders, global = True, ...)
  Sort(orders, global = False, ...)
```

Since there is no straightforward way to identify whether a node's output ordering is local or global, we should not remove a global sort even if its child is already ordered.

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
Unit tests

Closes #30093 from allisonwang-db/fix-sort.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-28 05:51:47 +00:00
Terry Kim 528160f001 [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `DROP TABLE` to use `UnresolvedTableOrView` to resolve the table/view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

### Why are the changes needed?

The current behavior is not consistent between v1 and v2 commands when resolving a temp view.
In v2, the `t` in the following example is resolved to a table:
```scala
sql("CREATE TABLE testcat.ns.t (id bigint) USING foo")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE testcat.ns")
sql("DROP TABLE t") // 't' is resolved to testcat.ns.t
```
whereas in v1, the `t` is resolved to a temp view:
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint) USING csv")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("DROP TABLE t") // 't' is resolved to a temp view
```

### Does this PR introduce _any_ user-facing change?

After this PR, for v2, `DROP TABLE t` is resolved to a temp view `t` instead of `testcat.ns.t`, consistent with v1 behavior.

### How was this patch tested?

Added a new test

Closes #30079 from imback82/drop_table_consistent.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-28 05:44:55 +00:00
Jungtaek Lim (HeartSaVioR) fcf8aa59b5 [SPARK-33240][SQL] Fail fast when fails to instantiate configured v2 session catalog
### What changes were proposed in this pull request?

This patch proposes to change the behavior on failing fast when Spark fails to instantiate configured v2 session catalog.

### Why are the changes needed?

The Spark behavior is against the intention of the end users - if end users configure session catalog which Spark would fail to initialize, Spark would swallow the error with only logging the error message and silently use the default catalog implementation.

This follows the voices on [discussion thread](https://lists.apache.org/thread.html/rdfa22a5ebdc4ac66e2c5c8ff0cd9d750e8a1690cd6fb456d119c2400%40%3Cdev.spark.apache.org%3E) in dev mailing list.

### Does this PR introduce _any_ user-facing change?

Yes. After the PR Spark will fail immediately if Spark fails to instantiate configured session catalog.

### How was this patch tested?

New UT added.

Closes #30147 from HeartSaVioR/SPARK-33240.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-28 03:31:11 +00:00
Ankur Dave 3f2a2b5fe6 [SPARK-33260][SQL] Fix incorrect results from SortExec when sortOrder is Stream
### What changes were proposed in this pull request?

The following query produces incorrect results. The query has two essential features: (1) it contains a string aggregate, resulting in a `SortExec` node, and (2) it contains a duplicate grouping key, causing `RemoveRepetitionFromGroupExpressions` to produce a sort order stored as a `Stream`.

```sql
SELECT bigint_col_1, bigint_col_9, MAX(CAST(bigint_col_1 AS string))
FROM table_4
GROUP BY bigint_col_1, bigint_col_9, bigint_col_9
```

When the sort order is stored as a `Stream`, the line `ordering.map(_.child.genCode(ctx))` in `GenerateOrdering#createOrderKeys()` produces unpredictable side effects to `ctx`. This is because `genCode(ctx)` modifies `ctx`. When ordering is a `Stream`, the modifications will not happen immediately as intended, but will instead occur lazily when the returned `Stream` is used later.

Similar bugs have occurred at least three times in the past: https://issues.apache.org/jira/browse/SPARK-24500, https://issues.apache.org/jira/browse/SPARK-25767, https://issues.apache.org/jira/browse/SPARK-26680.

The fix is to check if `ordering` is a `Stream` and force the modifications to happen immediately if so.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added a unit test for `SortExec` where `sortOrder` is a `Stream`. The test previously failed and now passes.

Closes #30160 from ankurdave/SPARK-33260.

Authored-by: Ankur Dave <ankurdave@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-27 13:20:22 -07:00
tanel.kiis@gmail.com 281f99c70b [SPARK-33225][SQL] Extract AliasHelper trait
### What changes were proposed in this pull request?

Extract methods related to handling Aliases to a trait.

### Why are the changes needed?

Avoid code duplication

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Existing UTs cover this

Closes #30134 from tanelk/SPARK-33225_aliasHelper.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@reach-u.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-27 22:53:05 +09:00
xuewei.linxuewei 537a49fc09 [SPARK-33140][SQL] remove SQLConf and SparkSession in all sub-class of Rule[QueryPlan]
### What changes were proposed in this pull request?

Since Issue [SPARK-33139](https://issues.apache.org/jira/browse/SPARK-33139) has been done, and SQLConf.get and SparkSession.active are more reliable. We are trying to refine the existing code usage of passing SQLConf and SparkSession into sub-class of Rule[QueryPlan].

In this PR.

* remove SQLConf from ctor-parameter of all sub-class of Rule[QueryPlan].
* using SQLConf.get to replace the original SQLConf instance.
* remove SparkSession from ctor-parameter of all sub-class of Rule[QueryPlan].
* using SparkSession.active to replace the original SparkSession instance.

### Why are the changes needed?

Code refine.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?

Existing test

Closes #30097 from leanken/leanken-SPARK-33140.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-27 12:40:57 +00:00
Cheng Su 1042d49bf9 [SPARK-33075][SQL] Enable auto bucketed scan by default (disable only for cached query)
### What changes were proposed in this pull request?

This PR is to enable auto bucketed table scan by default, with exception to only disable for cached query (similar to AQE). The reason why disabling auto scan for cached query is that, the cached query output partitioning can be leveraged later to avoid shuffle and sort when doing join and aggregate.

### Why are the changes needed?

Enable auto bucketed table scan by default is useful as it can optimize query automatically under the hood, without users interaction.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit test for cached query in `DisableUnnecessaryBucketedScanSuite.scala`. Also change a bunch of unit tests which should disable auto bucketed scan to make them work.

Closes #30138 from c21/enable-auto-bucket.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-26 20:23:24 +09:00
Yuning Zhang a21945ce6c [SPARK-33197][SQL] Make changes to spark.sql.analyzer.maxIterations take effect at runtime
### What changes were proposed in this pull request?

Make changes to `spark.sql.analyzer.maxIterations` take effect at runtime.

### Why are the changes needed?

`spark.sql.analyzer.maxIterations` is not a static conf. However, before this patch, changing `spark.sql.analyzer.maxIterations` at runtime does not take effect.

### Does this PR introduce _any_ user-facing change?

Yes. Before this patch, changing `spark.sql.analyzer.maxIterations` at runtime does not take effect.

### How was this patch tested?

modified unit test

Closes #30108 from yuningzh-db/dynamic-analyzer-max-iterations.

Authored-by: Yuning Zhang <yuning.zhang@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-26 16:19:06 +09:00
Cheng Su d87a0bb2ca [SPARK-32862][SS] Left semi stream-stream join
### What changes were proposed in this pull request?

This is to support left semi join in stream-stream join. The implementation of left semi join is (mostly in `StreamingSymmetricHashJoinExec` and `SymmetricHashJoinStateManager`):
* For left side input row, check if there's a match on right side state store.
  * if there's a match, output the left side row, but do not put the row in left side state store (no need to put in state store).
  * if there's no match, output nothing, but put the row in left side state store (with "matched" field to set to false in state store).
* For right side input row, check if there's a match on left side state store.
  * For all matched left rows in state store, output the rows with "matched" field as false. Set all left rows with "matched" field to be true. Only output the left side rows matched for the first time to guarantee left semi join semantics.
* State store eviction: evict rows from left/right side state store below watermark, same as inner join.

Note a followup optimization can be to evict matched left side rows from state store earlier, even when the rows are still above watermark. However this needs more change in `SymmetricHashJoinStateManager`, so will leave this as a followup.

### Why are the changes needed?

Current stream-stream join supports inner, left outer and right outer join (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L166 ). We do see internally a lot of users are using left semi stream-stream join (not spark structured streaming), e.g. I want to get the ad impression (join left side) which has click (joint right side), but I don't care how many clicks per ad (left semi semantics).

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit tests in `UnsupportedOperationChecker.scala` and `StreamingJoinSuite.scala`.

Closes #30076 from c21/stream-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-10-26 13:33:06 +09:00
Liang-Chi Hsieh 87b32f65ef [MINOR][DOCS][TESTS] Fix PLAN_CHANGE_LOG_LEVEL document
### What changes were proposed in this pull request?

`PLAN_CHANGE_LOG_LEVEL` config document is wrong. This is to fix it.

### Why are the changes needed?

Fix wrong doc.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Only doc change.

Closes #30136 from viirya/minor-sqlconf.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-23 13:35:46 +09:00
Ankit Srivastava 3819d39607 [SPARK-32998][BUILD] Add ability to override default remote repos with internal one
### What changes were proposed in this pull request?
- Building spark internally in orgs where access to outside internet is not allowed takes a long time because unsuccessful attempts are made to download artifacts from repositories which are not accessible. The unsuccessful attempts unnecessarily add significant amount of time to the build. I have seen a difference of up-to 1hr for some runs.
- Adding 1 environment variables that should be present that the start of the build and if they exist, override the default repos defined in the code and scripts.
envVariables:
      - DEFAULT_ARTIFACT_REPOSITORY=https://artifacts.internal.com/libs-release/

### Why are the changes needed?

To allow orgs to build spark internally without relying on external repositories for artifact downloads.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Multiple builds with and without env variables set.

Closes #29874 from ankits/SPARK-32998.

Authored-by: Ankit Srivastava <ankit_srivastava@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-22 16:35:55 -07:00
Xuedong Luan d9ee33cfb9 [SPARK-26533][SQL] Support query auto timeout cancel on thriftserver
### What changes were proposed in this pull request?

Support query auto cancelling when running too long on thriftserver.

This is the rework of #28991 and the credit should be the original author, leoluan2009.

Closes #28991

### Why are the changes needed?

For some cases, we use thriftserver as long-running applications.
Some times we want all the query need not to run more than given time.
In these cases, we can enable auto cancel for time-consumed query.Which can let us release resources for other queries to run.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added tests.

Closes #29933 from maropu/pr28991.

Lead-authored-by: Xuedong Luan <luanxuedong2009@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Luan <luanxuedong2009@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-22 17:23:10 +09:00
gengjiaan eb33bcb4b2 [SPARK-30796][SQL] Add parameter position for REGEXP_REPLACE
### What changes were proposed in this pull request?
`REGEXP_REPLACE` could replace all substrings of string that match regexp with replacement string.
But `REGEXP_REPLACE` lost some flexibility. such as: converts camel case strings to a string containing lower case words separated by an underscore:
AddressLine1 -> address_line_1
If we support the parameter position, we can do like this(e.g. Oracle):

```
WITH strings as (
  SELECT 'AddressLine1' s FROM dual union all
  SELECT 'ZipCode' s FROM dual union all
  SELECT 'Country' s FROM dual
)
  SELECT s "STRING",
         lower(regexp_replace(s, '([A-Z0-9])', '_\1', 2)) "MODIFIED_STRING"
  FROM strings;
```
The output:
```
  STRING               MODIFIED_STRING
-------------------- --------------------
AddressLine1         address_line_1
ZipCode              zip_code
Country              country
```

There are some mainstream database support the syntax.

**Oracle**
https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/REGEXP_REPLACE.html#GUID-EA80A33C-441A-4692-A959-273B5A224490

**Vertica**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/RegularExpressions/REGEXP_REPLACE.htm?zoom_highlight=regexp_replace

**Redshift**
https://docs.aws.amazon.com/redshift/latest/dg/REGEXP_REPLACE.html

### Why are the changes needed?
The parameter position for `REGEXP_REPLACE` is very useful.

### Does this PR introduce _any_ user-facing change?
'Yes'.

### How was this patch tested?
Jenkins test.

Closes #29891 from beliefer/add-position-for-regex_replace.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-22 07:59:49 +00:00
Chao Sun cb3fa6c936 [SPARK-33212][BUILD] Move to shaded clients for Hadoop 3.x profile
### What changes were proposed in this pull request?

This switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x. For Hadoop 2.7, we'll still use the same modules such as hadoop-client.

In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:

```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```

which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.

Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).

### Why are the changes needed?

This serves two purposes:
- to unblock Spark from upgrading to Hadoop 3.2.2/3.3.0+. Latest Hadoop versions have upgraded to use Guava 27+ and in order to adopt the latest Hadoop versions in Spark, we'll need to resolve the Guava conflicts. This takes the approach by switching to shaded client jars provided by Hadoop.
- avoid pulling 3rd party dependencies from Hadoop and avoid potential future conflicts.

### Does this PR introduce _any_ user-facing change?

When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.

### How was this patch tested?

Relying on existing tests.

Closes #29843 from sunchao/SPARK-29250.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-10-22 03:21:34 +00:00
Max Gekk ba13b94f6b [SPARK-33210][SQL] Set the rebasing mode for parquet INT96 type to EXCEPTION by default
### What changes were proposed in this pull request?
1. Set the default value for the SQL configs `spark.sql.legacy.parquet.int96RebaseModeInWrite` and `spark.sql.legacy.parquet.int96RebaseModeInRead` to `EXCEPTION`.
2. Update the SQL migration guide.

### Why are the changes needed?
Current default value `LEGACY` may lead to shifting timestamps in read or in write. We should leave the decision about rebasing to users.

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
By existing test suites like `ParquetIOSuite`.

Closes #30121 from MaxGekk/int96-exception-by-default.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-22 03:04:29 +00:00
Max Gekk a44e008de3 [SPARK-33160][SQL] Allow saving/loading INT96 in parquet w/o rebasing
### What changes were proposed in this pull request?
1. Add the SQL config `spark.sql.legacy.parquet.int96RebaseModeInWrite` to control timestamps rebasing in saving them as INT96. It supports the same set of values as `spark.sql.legacy.parquet.datetimeRebaseModeInWrite` but the default value is `LEGACY` to preserve backward compatibility with Spark <= 3.0.
2. Write the metadata key `org.apache.spark.int96NoRebase` to parquet files if the files are saved with `spark.sql.legacy.parquet.int96RebaseModeInWrite` isn't set to `LEGACY`.
3. Add the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` to control loading INT96 timestamps when parquet metadata doesn't have enough info (the `org.apache.spark.int96NoRebase` tag) about parquet writer - either INT96 was written by Proleptic Gregorian system or some Julian one.
4. Modified Vectorized and Parquet-mr Readers to support loading/saving INT96 timestamps w/o rebasing depending on SQL config and the metadata tag:
    - **No rebasing** in testing when the SQL config `spark.test.forceNoRebase` is set to `true`
    - **No rebasing** if parquet metadata contains the tag `org.apache.spark.int96NoRebase`. This is the case when parquet files are saved by Spark >= 3.1 with `spark.sql.legacy.parquet.datetimeRebaseModeInWrite` is set to `CORRECTED`, or saved by other systems with the tag `org.apache.spark.int96NoRebase`.
    - **With rebasing** if parquet files saved by Spark (any versions) without the metadata tag `org.apache.spark.int96NoRebase`.
    - Rebasing depend on the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` if there are no metadata tags `org.apache.spark.version` and `org.apache.spark.int96NoRebase`.

New SQL configs are added instead of re-using existing `spark.sql.legacy.parquet.datetimeRebaseModeInWrite` and `spark.sql.legacy.parquet.datetimeRebaseModeInRead` because of:
- To allow users have different modes for INT96 and for TIMESTAMP_MICROS (MILLIS). For example, users might want to save INT96 as LEGACY but TIMESTAMP_MICROS as CORRECTED.
- To have different modes for INT96 and DATE in load (or in save).
- To be backward compatible with Spark 2.4. For now, `spark.sql.legacy.parquet.datetimeRebaseModeInWrite/Read` are set to `EXCEPTION` by default.

### Why are the changes needed?
1. Parquet spec says that INT96 must be stored as Julian days (see https://github.com/apache/parquet-format/pull/49). This doesn't mean that a reader ( or a writer) is based on the Julian calendar. So, rebasing from Proleptic Gregorian to Julian calendar can be not needed.
2. Rebasing from/to Julian calendar can loose information because dates in one calendar don't exist in another one. Like 1582-10-04..1582-10-15 exist in Proleptic Gregorian calendar but not in the hybrid calendar (Julian + Gregorian), and visa versa, Julian date 1000-02-29 doesn't exist in Proleptic Gregorian calendar. We should allow users to save timestamps without loosing such dates (rebasing shifts such dates to the next valid date).
3. It would also make Spark compatible with other systems such as Impala and newer versions of Hive that write proleptic Gregorian based INT96 timestamps.

### Does this PR introduce _any_ user-facing change?
It can when `spark.sql.legacy.parquet.int96RebaseModeInWrite` is set non-default value `LEGACY`.

### How was this patch tested?
- Added a test to check the metadata key `org.apache.spark.int96NoRebase`
- By `ParquetIOSuite`

Closes #30056 from MaxGekk/parquet-rebase-int96.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-20 14:58:59 +09:00
Liang-Chi Hsieh 66c5e01322 [SPARK-32941][SQL] Optimize UpdateFields expression chain and put the rule early in Analysis phase
### What changes were proposed in this pull request?

This patch proposes to add more optimization to `UpdateFields` expression chain. And optimize `UpdateFields` early in analysis phase.

### Why are the changes needed?

`UpdateFields` can manipulate complex nested data, but using `UpdateFields` can easily create inefficient expression chain. We should optimize it further.

Because when manipulating deeply nested schema, the `UpdateFields` expression tree could be too complex to analyze, this change optimizes `UpdateFields` early in analysis phase.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test.

Closes #29812 from viirya/SPARK-32941.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-19 10:35:34 -07:00
tanel.kiis@gmail.com ce498943d2 [SPARK-33177][SQL] CollectList and CollectSet should not be nullable
### What changes were proposed in this pull request?

Mark `CollectList` and `CollectSet` as non-nullable.

### Why are the changes needed?

`CollectList` and `CollectSet` SQL expressions never return null value. Marking them as non-nullable can have some performance benefits, because some optimizer rules apply only to non-nullable expressions

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Did not find any existing tests on the nullability of aggregate functions.

Closes #30087 from tanelk/SPARK-33177_collect.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-19 09:50:59 +09:00
Liang-Chi Hsieh 3010e9044e [SPARK-33170][SQL] Add SQL config to control fast-fail behavior in FileFormatWriter
### What changes were proposed in this pull request?

This patch proposes to add a config we can control fast-fail behavior in FileFormatWriter and set it false by default.

### Why are the changes needed?

In SPARK-29649, we catch `FileAlreadyExistsException` in `FileFormatWriter` and fail fast for the task set to prevent task retry.

Due to latest discussion, it is important to be able to keep original behavior that is to retry tasks even `FileAlreadyExistsException` is thrown, because `FileAlreadyExistsException` could be recoverable in some cases.

We are going to add a config we can control this behavior and set it false for fast-fail by default.

### Does this PR introduce _any_ user-facing change?

Yes. By default the task in FileFormatWriter will retry even if `FileAlreadyExistsException` is thrown. This is the behavior before Spark 3.0. User can control fast-fail behavior by enabling it.

### How was this patch tested?

Unit test.

Closes #30073 from viirya/SPARK-33170.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-17 21:02:25 -07:00
Liang-Chi Hsieh 2c4599db4b [MINOR][SS][DOCS] Update Structured Streaming guide doc and update code typo
### What changes were proposed in this pull request?

This is a minor change to update structured-streaming-programming-guide and typos in code.

### Why are the changes needed?

Keep the user-facing document correct and updated.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Unit tests.

Closes #30074 from viirya/ss-minor.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-16 22:18:12 -07:00
Liang-Chi Hsieh e574fcd230 [SPARK-32376][SQL] Make unionByName null-filling behavior work with struct columns
### What changes were proposed in this pull request?

SPARK-29358 added support for `unionByName` to work when the two datasets didn't necessarily have the same schema, but it does not work with nested columns like structs. This patch adds the support to work with struct columns.

The behavior before this PR:

```scala
scala> val df1 = spark.range(1).selectExpr("id c0", "named_struct('c', id + 1, 'b', id + 2, 'a', id + 3) c1")
scala> val df2 = spark.range(1).selectExpr("id c0", "named_struct('c', id + 1, 'b', id + 2) c1")
scala> df1.unionByName(df2, true).printSchema
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. struct<c:bigint,b:bigint> <> struct<c:bigint,b:bigint,a:bigint> at the second column of the second table;;
'Union false, false
:- Project [id#0L AS c0#2L, named_struct(c, (id#0L + cast(1 as bigint)), b, (id#0L + cast(2 as bigint)), a, (id#0L + cast(3 as bigint))) AS c1#3]
:  +- Range (0, 1, step=1, splits=Some(12))
+- Project [c0#8L, c1#9]
   +- Project [id#6L AS c0#8L, named_struct(c, (id#6L + cast(1 as bigint)), b, (id#6L + cast(2 as bigint))) AS c1#9]
      +- Range (0, 1, step=1, splits=Some(12))
```

The behavior after this PR:

```scala
scala> df1.unionByName(df2, true).printSchema
root
 |-- c0: long (nullable = false)
 |-- c1: struct (nullable = false)
 |    |-- a: long (nullable = true)
 |    |-- b: long (nullable = false)
 |    |-- c: long (nullable = false)
scala> df1.unionByName(df2, true).show()
+---+-------------+
| c0|           c1|
+---+-------------+
|  0|    {3, 2, 1}|
|  0|{ null, 2, 1}|
+---+-------------+
```

### Why are the changes needed?

The `allowMissingColumns` of `unionByName` is a feature allowing merging different schema from two datasets when unioning them together. Nested column support makes the feature more general and flexible for usage.

### Does this PR introduce _any_ user-facing change?

Yes, after this change users can union two datasets with different schema with different structs.

### How was this patch tested?

Unit tests.

Closes #29587 from viirya/SPARK-32376.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-10-16 14:48:14 -07:00
ulysses 3ae1520185 [SPARK-33131][SQL] Fix grouping sets with having clause can not resolve qualified col name
### What changes were proposed in this pull request?

Correct the resolution of having clause.

### Why are the changes needed?

Grouping sets construct new aggregate lost the qualified name of grouping expression. Here is a example:
```
-- Works resolved by `ResolveReferences`
select c1 from values (1) as t1(c1) group by grouping sets(t1.c1) having c1 = 1

-- Works because of the extra expression c1
select c1 as c2 from values (1) as t1(c1) group by grouping sets(t1.c1) having t1.c1 = 1

-- Failed
select c1 from values (1) as t1(c1) group by grouping sets(t1.c1) having t1.c1 = 1
```

It wroks with `Aggregate` without grouping sets through `ResolveReferences`, but Grouping sets not works since the exprId has been changed.

### Does this PR introduce _any_ user-facing change?

Yes, bug fix.

### How was this patch tested?

add test.

Closes #30029 from ulysses-you/SPARK-33131.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-16 11:26:27 +00:00
gengjiaan b69e0651fe [SPARK-33126][SQL] Simplify offset window function(Remove direction field)
### What changes were proposed in this pull request?
The current `Lead`/`Lag` extends `OffsetWindowFunction`. `OffsetWindowFunction` contains field `direction` and use `direction` to calculates the `boundary`.

We can use single literal expression unify the two properties.
For example:
3 means `direction` is Asc and `boundary` is 3.
-3 means `direction` is Desc and `boundary` is -3.

### Why are the changes needed?
Improve the current implement of `Lead`/`Lag`.

### Does this PR introduce _any_ user-facing change?
 'No'.

### How was this patch tested?
Jenkins test.

Closes #30023 from beliefer/SPARK-33126.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-16 11:11:57 +00:00
xuewei.linxuewei 306872eefa [SPARK-33139][SQL] protect setActionSession and clearActiveSession
### What changes were proposed in this pull request?

This PR is a sub-task of [SPARK-33138](https://issues.apache.org/jira/browse/SPARK-33138). In order to make SQLConf.get reliable and stable, we need to make sure user can't pollute the SQLConf and SparkSession Context via calling setActiveSession and clearActiveSession.

Change of the PR:

* add legacy config spark.sql.legacy.allowModifyActiveSession to fallback to old behavior if user do need to call these two API.
* by default, if user call these two API, it will throw exception
* add extra two internal and private API setActiveSessionInternal and clearActiveSessionInternal for current internal usage
* change all internal reference to new internal API except for SQLContext.setActive and SQLContext.clearActive

### Why are the changes needed?

Make SQLConf.get reliable and stable.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?

* Add UT in SparkSessionBuilderSuite to test the legacy config
* Existing test

Closes #30042 from leanken/leanken-SPARK-33139.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-16 06:05:17 +00:00
Denis Pyshev ba69d68d91 [SPARK-33080][BUILD] Replace fatal warnings snippet
### What changes were proposed in this pull request?

Current solution in build file to enable build failure on compilation warnings with exclusion of deprecation ones is not portable after SBT version 1.3.13 (build import fails with compilation error with SBT 1.4) and could be replaced with more robust and maintainable, especially since Scala 2.13.2 with similar built-in functionality.

Additionally, warnings were fixed to pass the build, with as few changes as possible:
warnings in 2.12 compilation fixed in code,
warnings in 2.13 compilation covered by configuration to be addressed separately

### Why are the changes needed?

Unblocks upgrade to SBT after 1.3.13.
Enhances build file maintainability.
Allows fine tune of warnings configuration in scope of Scala 2.13 compilation.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

`build/sbt`'s `compile` and `Test/compile` for both Scala 2.12 and 2.13 profiles.

Closes #29995 from gemelen/feature/warnings-reporter.

Authored-by: Denis Pyshev <git@gemelen.net>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-10-15 14:49:43 -05:00
Liang-Chi Hsieh 9e3746469c [SPARK-33078][SQL] Add config for json expression optimization
### What changes were proposed in this pull request?

This proposes to add a config for json expression optimization.

### Why are the changes needed?

For the new Json expression optimization rules, it is safer if we can disable it using SQL config.

### Does this PR introduce _any_ user-facing change?

Yes, users can disable json expression optimization rule.

### How was this patch tested?

Unit test

Closes #30047 from viirya/SPARK-33078.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-10-15 12:38:10 -07:00
Jungtaek Lim (HeartSaVioR) 8e5cb1d276 [SPARK-33136][SQL] Fix mistakenly swapped parameter in V2WriteCommand.outputResolved
### What changes were proposed in this pull request?

This PR proposes to fix a bug on calling `DataType.equalsIgnoreCompatibleNullability` with mistakenly swapped parameters in `V2WriteCommand.outputResolved`. The order of parameters for `DataType.equalsIgnoreCompatibleNullability` are `from` and `to`, which says that the right order of matching variables are `inAttr` and `outAttr`.

### Why are the changes needed?

Spark throws AnalysisException due to unresolved operator in v2 write, while the operator is unresolved due to a bug that parameters to call `DataType.equalsIgnoreCompatibleNullability` in `outputResolved` have been swapped.

### Does this PR introduce _any_ user-facing change?

Yes, end users no longer suffer on unresolved operator in v2 write if they're trying to write dataframe containing non-nullable complex types against table matching complex types as nullable.

### How was this patch tested?

New UT added.

Closes #30033 from HeartSaVioR/SPARK-33136.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-14 08:30:03 -07:00
Max Gekk 05a62dcada [SPARK-33134][SQL] Return partial results only for root JSON objects
### What changes were proposed in this pull request?
In the PR, I propose to restrict the partial result feature only by root JSON objects. JSON datasource as well as `from_json()` will return `null` for malformed nested JSON objects.

### Why are the changes needed?
1. To not raise exception to users in the PERMISSIVE mode
2. To fix a regression and to have the same behavior as Spark 2.4.x has
3. Current implementation of partial result is supposed to work only for root (top-level) JSON objects, and not tested for bad nested complex JSON fields.

### Does this PR introduce _any_ user-facing change?
Yes. Before the changes, the code below:
```scala
    val pokerhand_raw = Seq("""[{"cards": [19], "playerId": 123456}]""").toDF("events")
    val event = new StructType().add("playerId", LongType).add("cards", ArrayType(new StructType().add("id", LongType).add("rank", StringType)))
    val pokerhand_events = pokerhand_raw.select(from_json($"events", ArrayType(event)).as("event"))
    pokerhand_events.show
```
throws the exception even in the default **PERMISSIVE** mode:
```java
java.lang.ClassCastException: java.lang.Long cannot be cast to org.apache.spark.sql.catalyst.util.ArrayData
  at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getArray(rows.scala:48)
  at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getArray$(rows.scala:48)
  at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getArray(rows.scala:195)
```

After the changes:
```
+-----+
|event|
+-----+
| null|
+-----+
```

### How was this patch tested?
Added a test to `JsonFunctionsSuite`.

Closes #30031 from MaxGekk/json-skip-row-wrong-schema.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-14 12:13:54 +09:00
xuewei.linxuewei dc697a8b59 [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 13:21:45 +00:00
gengjiaan 2b7239edfb [SPARK-33125][SQL] Improve the error when Lead and Lag are not allowed to specify window frame
### What changes were proposed in this pull request?
Except for Postgresql, other data sources (for example: vertica, oracle, redshift, mysql, presto) are not allowed to specify window frame for the Lead and Lag functions.

But the current error message is not clear enough.
`Window Frame $f must match the required frame`
This PR will use the following error message.
`Cannot specify window frame for lead function`

### Why are the changes needed?
Make clear error message.

### Does this PR introduce _any_ user-facing change?
Yes
Users will see the clearer error message.

### How was this patch tested?
Jenkins test.

Closes #30021 from beliefer/SPARK-33125.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-13 13:12:17 +00:00
Huaxin Gao af3e2f7d58 [SPARK-33081][SQL] Support ALTER TABLE in JDBC v2 Table Catalog: update type and nullability of columns (DB2 dialect)
### What changes were proposed in this pull request?
- Override the default SQL strings in the DB2 Dialect for:

  * ALTER TABLE UPDATE COLUMN TYPE
  * ALTER TABLE UPDATE COLUMN NULLABILITY

- Add new docker integration test suite jdbc/v2/DB2IntegrationSuite.scala

### Why are the changes needed?
In SPARK-24907, we implemented JDBC v2 Table Catalog but it doesn't support some ALTER TABLE at the moment. This PR supports DB2 specific ALTER TABLE.

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
By running new integration test suite:

$ ./build/sbt -Pdocker-integration-tests "test-only *.DB2IntegrationSuite"

Closes #29972 from huaxingao/db2_docker.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-13 12:57:54 +00:00
Chao Sun feee8da14b [SPARK-32858][SQL] UnwrapCastInBinaryComparison: support other numeric types
### What changes were proposed in this pull request?

In SPARK-24994 we implemented unwrapping cast for **integral types**. This extends it to support **numeric types** such as float/double/decimal, so that filters involving these types can be better pushed down to data sources.

Unlike the cases of integral types, conversions between numeric types can result to rounding up or downs. Consider the following case:

```sql
cast(e as double) < 1.9
```

assume type of `e` is short, since 1.9 is not representable in the type, the casting will either truncate or round. Now suppose the literal is truncated, we cannot convert the expression to:

```sql
e < cast(1.9 as short)
```

as in the previous implementation, since if `e` is 1, the original expression evaluates to true, but converted expression will evaluate to false.

To resolve the above, this PR first finds out whether casting from the wider type to the narrower type will result to truncate or round, by comparing a _roundtrip value_ derived from **converting the literal first to the narrower type, and then to the wider type**, versus the original literal value. For instance, in the above, we'll first obtain a roundtrip value via the conversion (double) 1.9 -> (short) 1 -> (double) 1.0, and then compare it against 1.9.

<img width="1153" alt="Screen Shot 2020-09-28 at 3 30 27 PM" src="https://user-images.githubusercontent.com/506679/94492719-bd29e780-019f-11eb-9111-71d6e3d157f7.png">

Now in the case of truncate, we'd convert the original expression to:
```sql
e <= cast(1.9 as short)
```
instead, so that the conversion also is valid when `e` is 1.

For more details, please check [this blog post](https://prestosql.io/blog/2019/05/21/optimizing-the-casts-away.html) by Presto which offers a very good explanation on how it works.

### Why are the changes needed?

For queries such as:
```sql
SELECT * FROM tbl WHERE short_col < 100.5
```
The predicate `short_col < 100.5` can't be pushed down to data sources because it involves casts. This eliminates the cast so these queries can run more efficiently.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit tests

Closes #29792 from sunchao/SPARK-32858.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-13 12:44:20 +00:00
tanel.kiis@gmail.com 17eebd7209 [SPARK-32295][SQL] Add not null and size > 0 filters before inner explode/inline to benefit from predicate pushdown
### What changes were proposed in this pull request?

Add `And(IsNotNull(e), GreaterThan(Size(e), Literal(0)))` filter before Explode, PosExplode and Inline, when `outer = false`.
Removed unused `InferFiltersFromConstraints` from `operatorOptimizationRuleSet` to avoid confusion that happened during the review process.

### Why are the changes needed?

Predicate pushdown will be able to move this new filter down through joins and into data sources for performance improvement.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test

Closes #29092 from tanelk/SPARK-32295.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@reach-u.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-13 20:11:04 +09:00
Takeshi Yamamuro a0e324460e [SPARK-32704][SQL][FOLLOWUP] Corrects version values of plan logging configs in SQLConf
### What changes were proposed in this pull request?

This PR intends to correct version values (`3.0.0` -> `3.1.0`) of three configs below in `SQLConf`:
 - spark.sql.planChangeLog.level
 - spark.sql.planChangeLog.rules
 - spark.sql.planChangeLog.batches

This PR comes from https://github.com/apache/spark/pull/29544#discussion_r503049350.

### Why are the changes needed?

Bugfix.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

N/A

Closes #30015 from maropu/pr29544-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-12 22:54:31 +09:00
Liang-Chi Hsieh 78c0967bbe [SPARK-33092][SQL] Support subexpression elimination in ProjectExec
### What changes were proposed in this pull request?

This patch proposes to add subexpression elimination support into `ProjectExec`. It can be controlled by `spark.sql.subexpressionElimination.enabled` config.

Before this change:

```scala
val df = spark.read.option("header", true).csv("/tmp/test.csv")
 df.withColumn("my_map", expr("str_to_map(foo, '&', '=')")).select(col("my_map")("foo"), col("my_map")("bar"), col("my_map")("baz")).debugCodegen
```

L27-40: first `str_to_map`.
L68:81: second `str_to_map`.
L109-122: third `str_to_map`.

```
/* 024 */   private void project_doConsume_0(InternalRow inputadapter_row_0, UTF8String project_expr_0_0, boolean project_exprIsNull_0_0) throws java.io.IOException {
/* 025 */     boolean project_isNull_0 = true;
/* 026 */     UTF8String project_value_0 = null;
/* 027 */     boolean project_isNull_1 = true;
/* 028 */     MapData project_value_1 = null;
/* 029 */
/* 030 */     if (!project_exprIsNull_0_0) {
/* 031 */       project_isNull_1 = false; // resultCode could change nullability.
/* 032 */
/* 033 */       UTF8String[] project_kvs_0 = project_expr_0_0.split(((UTF8String) references[1] /* literal */), -1);
/* 034 */       for(UTF8String kvEntry: project_kvs_0) {
/* 035 */         UTF8String[] kv = kvEntry.split(((UTF8String) references[2] /* literal */), 2);
/* 036 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[0] /* mapBuilder */).put(kv[0], kv.length == 2 ? kv[1] : null);
/* 037 */       }
/* 038 */       project_value_1 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[0] /* mapBuilder */).build();
/* 039 */
/* 040 */     }
/* 041 */     if (!project_isNull_1) {
/* 042 */       project_isNull_0 = false; // resultCode could change nullability.
/* 043 */
/* 044 */       final int project_length_0 = project_value_1.numElements();
/* 045 */       final ArrayData project_keys_0 = project_value_1.keyArray();
/* 046 */       final ArrayData project_values_0 = project_value_1.valueArray();
/* 047 */
/* 048 */       int project_index_0 = 0;
/* 049 */       boolean project_found_0 = false;
/* 050 */       while (project_index_0 < project_length_0 && !project_found_0) {
/* 051 */         final UTF8String project_key_0 = project_keys_0.getUTF8String(project_index_0);
/* 052 */         if (project_key_0.equals(((UTF8String) references[3] /* literal */))) {
/* 053 */           project_found_0 = true;
/* 054 */         } else {
/* 055 */           project_index_0++;
/* 056 */         }
/* 057 */       }
/* 058 */
/* 059 */       if (!project_found_0 || project_values_0.isNullAt(project_index_0)) {
/* 060 */         project_isNull_0 = true;
/* 061 */       } else {
/* 062 */         project_value_0 = project_values_0.getUTF8String(project_index_0);
/* 063 */       }
/* 064 */
/* 065 */     }
/* 066 */     boolean project_isNull_6 = true;
/* 067 */     UTF8String project_value_6 = null;
/* 068 */     boolean project_isNull_7 = true;
/* 069 */     MapData project_value_7 = null;
/* 070 */
/* 071 */     if (!project_exprIsNull_0_0) {
/* 072 */       project_isNull_7 = false; // resultCode could change nullability.
/* 073 */
/* 074 */       UTF8String[] project_kvs_1 = project_expr_0_0.split(((UTF8String) references[5] /* literal */), -1);
/* 075 */       for(UTF8String kvEntry: project_kvs_1) {
/* 076 */         UTF8String[] kv = kvEntry.split(((UTF8String) references[6] /* literal */), 2);
/* 077 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[4] /* mapBuilder */).put(kv[0], kv.length == 2 ? kv[1] : null);
/* 078 */       }
/* 079 */       project_value_7 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[4] /* mapBuilder */).build();
/* 080 */
/* 081 */     }
/* 082 */     if (!project_isNull_7) {
/* 083 */       project_isNull_6 = false; // resultCode could change nullability.
/* 084 */
/* 085 */       final int project_length_1 = project_value_7.numElements();
/* 086 */       final ArrayData project_keys_1 = project_value_7.keyArray();
/* 087 */       final ArrayData project_values_1 = project_value_7.valueArray();
/* 088 */
/* 089 */       int project_index_1 = 0;
/* 090 */       boolean project_found_1 = false;
/* 091 */       while (project_index_1 < project_length_1 && !project_found_1) {
/* 092 */         final UTF8String project_key_1 = project_keys_1.getUTF8String(project_index_1);
/* 093 */         if (project_key_1.equals(((UTF8String) references[7] /* literal */))) {
/* 094 */           project_found_1 = true;
/* 095 */         } else {
/* 096 */           project_index_1++;
/* 097 */         }
/* 098 */       }
/* 099 */
/* 100 */       if (!project_found_1 || project_values_1.isNullAt(project_index_1)) {
/* 101 */         project_isNull_6 = true;
/* 102 */       } else {
/* 103 */         project_value_6 = project_values_1.getUTF8String(project_index_1);
/* 104 */       }
/* 105 */
/* 106 */     }
/* 107 */     boolean project_isNull_12 = true;
/* 108 */     UTF8String project_value_12 = null;
/* 109 */     boolean project_isNull_13 = true;
/* 110 */     MapData project_value_13 = null;
/* 111 */
/* 112 */     if (!project_exprIsNull_0_0) {
/* 113 */       project_isNull_13 = false; // resultCode could change nullability.
/* 114 */
/* 115 */       UTF8String[] project_kvs_2 = project_expr_0_0.split(((UTF8String) references[9] /* literal */), -1);
/* 116 */       for(UTF8String kvEntry: project_kvs_2) {
/* 117 */         UTF8String[] kv = kvEntry.split(((UTF8String) references[10] /* literal */), 2);
/* 118 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[8] /* mapBuilder */).put(kv[0], kv.length == 2 ? kv[1] : null);
/* 119 */       }
/* 120 */       project_value_13 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[8] /* mapBuilder */).build();
/* 121 */
/* 122 */     }
...
```
After this change:

L27-40 evaluates the common map variable.

```
/* 024 */   private void project_doConsume_0(InternalRow inputadapter_row_0, UTF8String project_expr_0_0, boolean project_exprIsNull_0_0) throws java.io.IOException {
/* 025 */     // common sub-expressions
/* 026 */
/* 027 */     boolean project_isNull_0 = true;
/* 028 */     MapData project_value_0 = null;
/* 029 */
/* 030 */     if (!project_exprIsNull_0_0) {
/* 031 */       project_isNull_0 = false; // resultCode could change nullability.
/* 032 */
/* 033 */       UTF8String[] project_kvs_0 = project_expr_0_0.split(((UTF8String) references[1] /* literal */), -1);
/* 034 */       for(UTF8String kvEntry: project_kvs_0) {
/* 035 */         UTF8String[] kv = kvEntry.split(((UTF8String) references[2] /* literal */), 2);
/* 036 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[0] /* mapBuilder */).put(kv[0], kv.length == 2 ? kv[1] : null);
/* 037 */       }
/* 038 */       project_value_0 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[0] /* mapBuilder */).build();
/* 039 */
/* 040 */     }
/* 041 */
/* 042 */     boolean project_isNull_4 = true;
/* 043 */     UTF8String project_value_4 = null;
/* 044 */
/* 045 */     if (!project_isNull_0) {
/* 046 */       project_isNull_4 = false; // resultCode could change nullability.
/* 047 */
/* 048 */       final int project_length_0 = project_value_0.numElements();
/* 049 */       final ArrayData project_keys_0 = project_value_0.keyArray();
/* 050 */       final ArrayData project_values_0 = project_value_0.valueArray();
/* 051 */
/* 052 */       int project_index_0 = 0;
/* 053 */       boolean project_found_0 = false;
/* 054 */       while (project_index_0 < project_length_0 && !project_found_0) {
/* 055 */         final UTF8String project_key_0 = project_keys_0.getUTF8String(project_index_0);
/* 056 */         if (project_key_0.equals(((UTF8String) references[3] /* literal */))) {
/* 057 */           project_found_0 = true;
/* 058 */         } else {
/* 059 */           project_index_0++;
/* 060 */         }
/* 061 */       }
/* 062 */
/* 063 */       if (!project_found_0 || project_values_0.isNullAt(project_index_0)) {
/* 064 */         project_isNull_4 = true;
/* 065 */       } else {
/* 066 */         project_value_4 = project_values_0.getUTF8String(project_index_0);
/* 067 */       }
/* 068 */
/* 069 */     }
/* 070 */     boolean project_isNull_6 = true;
/* 071 */     UTF8String project_value_6 = null;
/* 072 */
/* 073 */     if (!project_isNull_0) {
/* 074 */       project_isNull_6 = false; // resultCode could change nullability.
/* 075 */
/* 076 */       final int project_length_1 = project_value_0.numElements();
/* 077 */       final ArrayData project_keys_1 = project_value_0.keyArray();
/* 078 */       final ArrayData project_values_1 = project_value_0.valueArray();
/* 079 */
/* 080 */       int project_index_1 = 0;
/* 081 */       boolean project_found_1 = false;
/* 082 */       while (project_index_1 < project_length_1 && !project_found_1) {
/* 083 */         final UTF8String project_key_1 = project_keys_1.getUTF8String(project_index_1);
/* 084 */         if (project_key_1.equals(((UTF8String) references[4] /* literal */))) {
/* 085 */           project_found_1 = true;
/* 086 */         } else {
/* 087 */           project_index_1++;
/* 088 */         }
/* 089 */       }
/* 090 */
/* 091 */       if (!project_found_1 || project_values_1.isNullAt(project_index_1)) {
/* 092 */         project_isNull_6 = true;
/* 093 */       } else {
/* 094 */         project_value_6 = project_values_1.getUTF8String(project_index_1);
/* 095 */       }
/* 096 */
/* 097 */     }
/* 098 */     boolean project_isNull_8 = true;
/* 099 */     UTF8String project_value_8 = null;
/* 100 */
...
```

When the code is split into separated method:

```
/* 026 */   private void project_doConsume_0(InternalRow inputadapter_row_0, UTF8String project_expr_0_0, boolean project_exprIsNull_0_0) throws java.io.IOException {
/* 027 */     // common sub-expressions
/* 028 */
/* 029 */     MapData project_subExprValue_0 = project_subExpr_0(project_exprIsNull_0_0, project_expr_0_0);
/* 030 */
...
/* 140 */   private MapData project_subExpr_0(boolean project_exprIsNull_0_0, org.apache.spark.unsafe.types.UTF8String project_expr_0_0) {
/* 141 */     boolean project_isNull_0 = true;
/* 142 */     MapData project_value_0 = null;
/* 143 */
/* 144 */     if (!project_exprIsNull_0_0) {
/* 145 */       project_isNull_0 = false; // resultCode could change nullability.
/* 146 */
/* 147 */       UTF8String[] project_kvs_0 = project_expr_0_0.split(((UTF8String) references[1] /* literal */), -1);
/* 148 */       for(UTF8String kvEntry: project_kvs_0) {
/* 149 */         UTF8String[] kv = kvEntry.split(((UTF8String) references[2] /* literal */), 2);
/* 150 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[0] /* mapBuilder */).put(kv[0], kv.length == 2 ? kv[1] : null);
/* 151 */       }
/* 152 */       project_value_0 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[0] /* mapBuilder */).build();
/* 153 */
/* 154 */     }
/* 155 */     project_subExprIsNull_0 = project_isNull_0;
/* 156 */     return project_value_0;
/* 157 */   }
```

### Why are the changes needed?

Users occasionally write repeated expression in projection. It is also possibly that query optimizer optimizes a query to evaluate same expression many times in a Project. Currently in ProjectExec, we don't support subexpression elimination in Whole-stage codegen. We can support it to reduce redundant evaluation.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

`spark.sql.subexpressionElimination.enabled` is enabled by default. So that's said we should pass all tests with this change.

Closes #29975 from viirya/SPARK-33092.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-12 16:54:21 +09:00
Gabor Somogyi 4af1ac9384 [SPARK-32047][SQL] Add JDBC connection provider disable possibility
### What changes were proposed in this pull request?
At the moment there is no possibility to turn off JDBC authentication providers which exists on the classpath. This can be problematic because service providers are loaded with service loader. In this PR I've added `spark.sql.sources.disabledJdbcConnProviderList` configuration possibility (default: empty).

### Why are the changes needed?
No possibility to turn off JDBC authentication providers.

### Does this PR introduce _any_ user-facing change?
Yes, it introduces new configuration option.

### How was this patch tested?
* Existing + newly added unit tests.
* Existing integration tests.

Closes #29964 from gaborgsomogyi/SPARK-32047.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-12 12:24:54 +09:00
Jungtaek Lim (HeartSaVioR) edb140eb5c [SPARK-32896][SS] Add DataStreamWriter.table API
### What changes were proposed in this pull request?

This PR proposes to add `DataStreamWriter.table` to specify the output "table" to write from the streaming query.

### Why are the changes needed?

For now, there's no way to write to the table (especially catalog table) even the table is capable to handle streaming write, so even with Spark 3, writing to the catalog table via SS should go through the `DataStreamWriter.format(provider)` and wish the provider can handle it as same as we do with catalog table.

With the new API, we can directly point to the catalog table which supports streaming write. Some of usages are covered with tests - simply saying, end users can do the following:

```scala
// assuming `testcat` is a custom catalog, and `ns` is a namespace in the catalog
spark.sql("CREATE TABLE testcat.ns.table1 (id bigint, data string) USING foo")

val query = inputDF
      .writeStream
      .table("testcat.ns.table1")
      .option(...)
      .start()
```

### Does this PR introduce _any_ user-facing change?

Yes, as this adds a new public API in DataStreamWriter. This doesn't bring backward incompatible change.

### How was this patch tested?

New unit tests.

Closes #29767 from HeartSaVioR/SPARK-32896.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-09 03:01:54 -07:00
ulysses a9077299d7 [SPARK-32743][SQL] Add distinct info at UnresolvedFunction toString
### What changes were proposed in this pull request?

Add distinct info at `UnresolvedFunction.toString`.

### Why are the changes needed?

Make `UnresolvedFunction` info complete.

```
create table test (c1 int, c2 int);
explain extended select sum(distinct c1) from test;

-- before this pr
== Parsed Logical Plan ==
'Project [unresolvedalias('sum('c1), None)]
+- 'UnresolvedRelation [test]

-- after this pr
== Parsed Logical Plan ==
'Project [unresolvedalias('sum(distinct 'c1), None)]
+- 'UnresolvedRelation [test]
```

### Does this PR introduce _any_ user-facing change?

Yes, get distinct info during sql parse.

### How was this patch tested?

manual test.

Closes #29586 from ulysses-you/SPARK-32743.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-09 09:25:22 +09:00
Max Gekk 7d6e3fb998 [SPARK-33074][SQL] Classify dialect exceptions in JDBC v2 Table Catalog
### What changes were proposed in this pull request?
1. Add new method to the `JdbcDialect` class - `classifyException()`. It converts dialect specific exception to Spark's `AnalysisException` or its sub-classes.
2. Replace H2 exception  `org.h2.jdbc.JdbcSQLException` in `JDBCTableCatalogSuite` by `AnalysisException`.
3. Add `H2Dialect`

### Why are the changes needed?
Currently JDBC v2 Table Catalog implementation throws dialect specific exception and ignores exceptions defined in the `TableCatalog` interface. This PR adds new method for converting dialect specific exception, and assumes that follow up PRs will implement `classifyException()`.

### Does this PR introduce _any_ user-facing change?
Yes.

### How was this patch tested?
By running existing test suites `JDBCTableCatalogSuite` and `JDBCV2Suite`.

Closes #29952 from MaxGekk/jdbcv2-classify-exception.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-08 05:28:33 +00:00
Karen Feng 39510b0e9b [SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true
## What changes were proposed in this pull request?

Adds a SQL function `raise_error` which underlies the refactored `assert_true` function. `assert_true` now also (optionally) accepts a custom error message field.
`raise_error` is exposed in SQL, Python, Scala, and R.
`assert_true` was previously only exposed in SQL; it is now also exposed in Python, Scala, and R.

### Why are the changes needed?

Improves usability of `assert_true` by clarifying error messaging, and adds the useful helper function `raise_error`.

### Does this PR introduce _any_ user-facing change?

Yes:
- Adds `raise_error` function to the SQL, Python, Scala, and R APIs.
- Adds `assert_true` function to the SQL, Python and R APIs.

### How was this patch tested?

Adds unit tests in SQL, Python, Scala, and R for `assert_true` and `raise_error`.

Closes #29947 from karenfeng/spark-32793.

Lead-authored-by: Karen Feng <karen.feng@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-08 12:05:39 +09:00
Takeshi Yamamuro 94d648dff5 [SPARK-33036][SQL] Refactor RewriteCorrelatedScalarSubquery code to replace exprIds in a bottom-up manner
### What changes were proposed in this pull request?

This PR intends to refactor code in `RewriteCorrelatedScalarSubquery` for replacing `ExprId`s in a bottom-up manner instead of doing in a top-down one.

This PR comes from the talk with cloud-fan in https://github.com/apache/spark/pull/29585#discussion_r490371252.

### Why are the changes needed?

To improve code.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #29913 from maropu/RefactorRewriteCorrelatedScalarSubquery.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-10-07 20:16:40 +09:00
Terry Kim 7e99fcd64e [SPARK-33004][SQL] Migrate DESCRIBE column to use UnresolvedTableOrView to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate `DESCRIBE tbl colname` to use `UnresolvedTableOrView` to resolve the table/view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

### Why are the changes needed?

The current behavior is not consistent between v1 and v2 commands when resolving a temp view.
In v2, the `t` in the following example is resolved to a table:
```scala
sql("CREATE TABLE testcat.ns.t (id bigint) USING foo")
sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i")
sql("USE testcat.ns")
sql("DESCRIBE t i") // 't' is resolved to testcat.ns.t

Describing columns is not supported for v2 tables.;
org.apache.spark.sql.AnalysisException: Describing columns is not supported for v2 tables.;
```
whereas in v1, the `t` is resolved to a temp view:
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint) USING csv")
sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i")
sql("USE spark_catalog.test")
sql("DESCRIBE t i").show // 't' is resolved to a temp view

+---------+----------+
|info_name|info_value|
+---------+----------+
| col_name|         i|
|data_type|       int|
|  comment|      NULL|
+---------+----------+
```

### Does this PR introduce _any_ user-facing change?

After this PR, `DESCRIBE t i` is resolved to a temp view `t` instead of `testcat.ns.t`.

### How was this patch tested?

Added a new test

Closes #29880 from imback82/describe_column_consistent.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-10-07 06:33:20 +00:00