Commit graph

8532 commits

Author SHA1 Message Date
HyukjinKwon 7cc0f0e9a7 [SPARK-28894][SQL][TESTS] Add a clue to make it easier to debug via Jenkins's test results
### What changes were proposed in this pull request?

See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/109834/testReport/junit/org.apache.spark.sql/SQLQueryTestSuite/

![Screen Shot 2019-08-28 at 4 08 58 PM](https://user-images.githubusercontent.com/6477701/63833484-2a23ea00-c9ae-11e9-91a1-0859cb183fea.png)

```xml
<?xml version="1.0" encoding="UTF-8"?>
<testsuite hostname="C02Y52ZLJGH5" name="org.apache.spark.sql.SQLQueryTestSuite" tests="3" errors="0" failures="0" skipped="0" time="14.475">
    ...
    <testcase classname="org.apache.spark.sql.SQLQueryTestSuite" name="sql - Scala UDF" time="6.703">
    </testcase>
    <testcase classname="org.apache.spark.sql.SQLQueryTestSuite" name="sql - Regular Python UDF" time="4.442">
    </testcase>
    <testcase classname="org.apache.spark.sql.SQLQueryTestSuite" name="sql - Scalar Pandas UDF" time="3.33">
    </testcase>
    <system-out/>
    <system-err/>
</testsuite>
```

Root cause seems a bug in SBT - it truncates the test name based on the last dot.

https://github.com/sbt/sbt/issues/2949
https://github.com/sbt/sbt/blob/v0.13.18/testing/src/main/scala/sbt/JUnitXmlTestsListener.scala#L71-L79

I tried to find a better way but couldn't find. Therefore, this PR proposes a workaround by appending the test file name into the assert log:

```diff
  [info] - inner-join.sql *** FAILED *** (4 seconds, 306 milliseconds)
+ [info]   inner-join.sql
  [info]   Expected "1	a
  [info]   1	a
  [info]   1	b
  [info]   1[]", but got "1	a
  [info]   1	a
  [info]   1	b
  [info]   1[	b]" Result did not match for query #6
  [info]   SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag (SQLQueryTestSuite.scala:377)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:528)
```

It will at least prevent us to search full logs to identify which test file is failed by clicking filed test.

Note that this PR does not fully fix the issue but only fix the logs on its failed tests.

### Why are the changes needed?
To debug Jenkins logs easier. Otherwise, we should open full logs and search which test was failed.

### Does this PR introduce any user-facing change?
It will print out the file name of failed tests in Jenkins' test reports.

### How was this patch tested?
Manually tested but Jenkins tests are required in this PR.

Now it at least shows which file it is:

![Screen Shot 2019-08-30 at 10 16 32 PM](https://user-images.githubusercontent.com/6477701/64023705-de22a200-cb73-11e9-8806-2e98ad35adef.png)

Closes #25630 from HyukjinKwon/SPARK-28894-1.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-30 15:10:40 -07:00
younggyu chun 3b07a4eb28 [SPARK-27931][SQL] Accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type
## What changes were proposed in this pull request?
This PR aims to add "true", "yes", "1", "false", "no", "0", and unique prefixes as input for the boolean data type and ignore input whitespace. Please see the following what string representations are using for the boolean type in other databases.

https://www.postgresql.org/docs/devel/datatype-boolean.html
https://docs.aws.amazon.com/redshift/latest/dg/r_Boolean_type.html

## How was this patch tested?
Added new tests to CastSuite.

Closes #25458 from younggyuchun/SPARK-27931.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-30 14:18:13 -07:00
Burak Yavuz 827969399b [SPARK-28668][SQL] Support V2SessionCatalog for ALTER TABLE
### What changes were proposed in this pull request?

Adds support for the V2SessionCatalog for ALTER TABLE statements.
Implementation changes are ~50 loc. The rest is just test refactoring.

### Why are the changes needed?
To allow V2 DataSources to plug in through a configurable plugin interface without requiring the explicit use of catalog identifiers, and leverage ALTER TABLE statements.

### How was this patch tested?

By re-using existing tests in DataSourceV2SQLSuite.

Closes #25502 from brkyvz/alterV3.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-30 14:16:47 +08:00
Wenchen Fan f8f7c52f12 [SPARK-28899][SQL][TEST] merge the testing in-memory v2 catalogs from catalyst and core
### What changes were proposed in this pull request?

There are 2 in-memory `TableCatalog` and `Table` implementations for testing, in sql/catalyst and sql/core. This PR merges them.

After merging, there are 3 classes:
1. `InMemoryTable`
2. `InMemoryTableCatalog`
3. `StagingInMemoryTableCatalog`

For better maintainability, these 3 classes are put in 3 different files.

### Why are the changes needed?

reduce duplicated code

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

no
### How was this patch tested?

N/A

Closes #25610 from cloud-fan/dsv2-test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Ryan Blue <blue@apache.org>
2019-08-29 12:56:19 -07:00
Gengliang Wang 24655583f1 [SPARK-28495][SQL][FOLLOW-UP] Disallow conversions between timestamp and long in ASNI mode
### What changes were proposed in this pull request?

Disallow conversions between `timestamp` type and `long` type in table insertion with ANSI store assignment policy.

### Why are the changes needed?

In the PR https://github.com/apache/spark/pull/25581, timestamp type is allowed to be converted to long type, since timestamp type is represented by long type internally, and both legacy mode and strict mode allows the conversion.

After reconsideration, I think we should disallow it. As per ANSI SQL section "4.4.2 Characteristics of numbers":
> A number is assignable only to sites of numeric type.

In PostgreSQL, the conversion between timestamp and long is also disallowed.

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

Conversion between timestamp and long is disallowed in table insertion with ANSI store assignment policy.

### How was this patch tested?

Unit test

Closes #25615 from gengliangwang/disallowTimeStampToLong.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-29 19:59:24 +08:00
Matt Hawes 137b20b964 [SPARK-28818][SQL] Respect source column nullability in the arrays created by freqItems()
### What changes were proposed in this pull request?
This PR replaces the hard-coded non-nullability of the array elements returned by `freqItems()` with a nullability that reflects the original schema. Essentially [the functional change](https://github.com/apache/spark/pull/25575/files#diff-bf59bb9f3dc351f5bf6624e5edd2dcf4R122) to the schema generation is:
```
StructField(name + "_freqItems", ArrayType(dataType, false))
```
Becomes:
```
StructField(name + "_freqItems", ArrayType(dataType, originalField.nullable))
```

Respecting the original nullability prevents issues when Spark depends on `ArrayType`'s `containsNull` being accurate. The example that uncovered this is calling `collect()` on the dataframe (see [ticket](https://issues.apache.org/jira/browse/SPARK-28818) for full repro). Though it's likely that there a several places where this could cause a problem.

I've also refactored a small amount of the surrounding code to remove some unnecessary steps and group together related operations.

### Why are the changes needed?
I think it's pretty clear why this change is needed. It fixes a bug that currently prevents users from calling `df.freqItems.collect()` along with potentially causing other, as yet unknown, issues.

### Does this PR introduce any user-facing change?
Nullability of columns when calling freqItems on them is now respected after the change.

### How was this patch tested?
I added a test that specifically tests the carry-through of the nullability as well as explicitly calling `collect()` to catch the exact regression that was observed. I also ran the test against the old version of the code and it fails as expected.

Closes #25575 from MGHawes/mhawes/SPARK-28818.

Authored-by: Matt Hawes <mhawes@palantir.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-29 10:49:10 +09:00
Yuming Wang 1b404b9b99 [SPARK-28890][SQL] Upgrade Hive Metastore Client to the 3.1.2 for Hive 3.1
### What changes were proposed in this pull request?

Hive 3.1.2 has been released. This PR upgrades the Hive Metastore Client to 3.1.2 for Hive 3.1.

Hive 3.1.2 release notes:
https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12344397&styleName=Html&projectId=12310843

### Why are the changes needed?

This is an improvement to support a newly release 3.1.2. Otherwise, it will throws `UnsupportedOperationException` if user `set spark.sql.hive.metastore.version=3.1.2`:
```scala
Exception in thread "main" java.lang.UnsupportedOperationException: Unsupported Hive Metastore version (3.1.2). Please set spark.sql.hive.metastore.version with a valid version.
	at org.apache.spark.sql.hive.client.IsolatedClientLoader$.hiveVersion(IsolatedClientLoader.scala:109)
```

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

### How was this patch tested?
Existing UT

Closes #25604 from wangyum/SPARK-28890.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-28 09:16:54 -07:00
Gengliang Wang 9d6bec183c [SPARK-28730][SPARK-28495][SQL][FOLLOW-UP] Revise the doc of option spark.sql.storeAssignmentPolicy
### What changes were proposed in this pull request?

Revise the documentation of SQL option `spark.sql.storeAssignmentPolicy`.

### Why are the changes needed?

1. Need to point out the ANSI mode is mostly the same with PostgreSQL
2. Need to point out Legacy mode allows type coercion as long as it is valid casting
3. Better examples.

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

No

### How was this patch tested?

Uni test

Closes #25605 from gengliangwang/reviseDoc.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-28 19:59:53 +08:00
Yuming Wang e3b32da027 [SPARK-25474][SQL][DOCS] Update the docs for spark.sql.statistics.fallBackToHdfs
## What changes were proposed in this pull request?

This PR update `spark.sql.statistics.fallBackToHdfs`'s doc:
1. This flag is effective only if it is Hive table.
2. For non-partitioned data source table, it will be automatically recalculated if table statistics are not available
3. For partitioned data source table, It is 'spark.sql.defaultSizeInBytes' if table statistics are not available.

Related code:
- Non-partitioned data source table:
[SizeInBytesOnlyStatsPlanVisitor.default()](98be8953c7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L54-L57)) -> [LogicalRelation.computeStats()](a1c1dd3484/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala (L42-L46)) -> [HadoopFsRelation.sizeInBytes()](c0632cec04/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala (L72-L75)) -> [PartitioningAwareFileIndex.sizeInBytes()](b276788d57/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala (L103))
`PartitioningAwareFileIndex.sizeInBytes()` is calculated by [`allFiles().map(_.getLen).sum`](b276788d57/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala (L103)) if table statistics are not available.

- Partitioned data source table:
[SizeInBytesOnlyStatsPlanVisitor.default()](98be8953c7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L54-L57)) -> [LogicalRelation.computeStats()](a1c1dd3484/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala (L42-L46)) -> [CatalogFileIndex.sizeInBytes](5d672b7f3e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala (L41))
`CatalogFileIndex.sizeInBytes` is [spark.sql.defaultSizeInBytes](c30b5297bc/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L387)) if table statistics are not available.

## How was this patch tested?

N/A

Closes #24715 from wangyum/SPARK-25474.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-28 19:15:26 +08:00
hemanth meka 6252c54e39 [SPARK-23519][SQL] create view should work from query with duplicate output columns
**What changes were proposed in this pull request?**

Moving the call for checkColumnNameDuplication out of generateViewProperties. This way we can choose ifcheckColumnNameDuplication will be performed on analyzed or aliased plan without having to pass an additional argument(aliasedPlan) to generateViewProperties.

Before the pr column name duplication was performed on the query output of below sql(c1, c1) and the pr makes it perform check on the user provided schema of view definition(c1, c2)

**Why are the changes needed?**

Changes are to fix SPARK-23519 bug. Below queries would cause an exception. This pr fixes them and also added a test case.

`CREATE TABLE t23519 AS SELECT 1 AS c1
CREATE VIEW v23519 (c1, c2) AS SELECT c1, c1 FROM t23519`

**Does this PR introduce any user-facing change?**
No

**How was this patch tested?**
new unit test added in SQLViewSuite

Closes #25570 from hem1891/SPARK-23519.

Lead-authored-by: hemanth meka <hmeka@tibco.com>
Co-authored-by: hem1891 <hem1891@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-28 12:11:10 +08:00
Wenchen Fan 90b10b4f7a [HOT-FIX] fix compilation
This is caused by 2 PRs that were merged at the same time:
cb06209fc9
2b24a71fec

Closes #25597 from cloud-fan/hot-fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 23:30:44 +08:00
Gengliang Wang 2b24a71fec [SPARK-28495][SQL] Introduce ANSI store assignment policy for table insertion
### What changes were proposed in this pull request?
 Introduce ANSI store assignment policy for table insertion.
With ANSI policy, Spark performs the type coercion of table insertion as per ANSI SQL.

### Why are the changes needed?
In Spark version 2.4 and earlier, when inserting into a table, Spark will cast the data type of input query to the data type of target table by coercion. This can be super confusing, e.g. users make a mistake and write string values to an int column.

In data source V2, by default, only upcasting is allowed when inserting data into a table. E.g. int -> long and int -> string are allowed, while decimal -> double or long -> int are not allowed. The rules of UpCast was originally created for Dataset type coercion. They are quite strict and different from the behavior of all existing popular DBMS. This is breaking change. It is possible that existing queries are broken after 3.0 releases.

Following ANSI SQL standard makes Spark consistent with the table insertion behaviors of popular DBMS like PostgreSQL/Oracle/Mysql.

### Does this PR introduce any user-facing change?
A new optional mode for table insertion.

### How was this patch tested?
Unit test

Closes #25581 from gengliangwang/ANSImode.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 22:13:23 +08:00
WeichenXu 7f605f5559 [SPARK-28621][SQL] Make spark.sql.crossJoin.enabled default value true
### What changes were proposed in this pull request?

Make `spark.sql.crossJoin.enabled` default value true

### Why are the changes needed?

For implicit cross join, we can set up a watchdog to cancel it if running for a long time.
When "spark.sql.crossJoin.enabled" is false, because `CheckCartesianProducts` is implemented in logical plan stage, it may generate some mismatching error which may confuse end user:
* it's done in logical phase, so we may fail queries that can be executed via broadcast join, which is very fast.
* if we move the check to the physical phase, then a query may success at the beginning, and begin to fail when the table size gets larger (other people insert data to the table). This can be quite confusing.
* the CROSS JOIN syntax doesn't work well if join reorder happens.
* some non-equi-join will generate plan using cartesian product, but `CheckCartesianProducts` do not detect it and raise error.

So that in order to address this in simpler way, we can turn off showing this cross-join error by default.

For reference, I list some cases raising mismatching error here:
Providing:
```
spark.range(2).createOrReplaceTempView("sm1") // can be broadcast
spark.range(50000000).createOrReplaceTempView("bg1") // cannot be broadcast
spark.range(60000000).createOrReplaceTempView("bg2") // cannot be broadcast
```
1) Some join could be convert to broadcast nested loop join, but CheckCartesianProducts raise error. e.g.
```
select sm1.id, bg1.id from bg1 join sm1 where sm1.id < bg1.id
```
2) Some join will run by CartesianJoin but CheckCartesianProducts DO NOT raise error. e.g.
```
select bg1.id, bg2.id from bg1 join bg2 where bg1.id < bg2.id
```

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

### How was this patch tested?

Closes #25520 from WeichenXu123/SPARK-28621.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 21:53:37 +08:00
Yuming Wang e12da8b957 [SPARK-28876][SQL] fallBackToHdfs should not support Hive partitioned table
### What changes were proposed in this pull request?

This PR makes `spark.sql.statistics.fallBackToHdfs` not support Hive partitioned tables.

### Why are the changes needed?

The current implementation is incorrect for external partitions and it is expensive to support partitioned table with external partitions.

### Does this PR introduce any user-facing change?
Yes.  But I think it will not change the join strategy because partitioned table usually very large.

### How was this patch tested?
unit test

Closes #25584 from wangyum/SPARK-28876.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 21:37:18 +08:00
Yuming Wang 96179732aa [SPARK-27592][SQL][TEST][FOLLOW-UP] Test set the partitioned bucketed data source table SerDe correctly
### What changes were proposed in this pull request?
This PR add test for set the partitioned bucketed data source table SerDe correctly.

### Why are the changes needed?
Improve test.

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

### How was this patch tested?
N/A

Closes #25591 from wangyum/SPARK-27592-f1.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 21:10:58 +08:00
Wenchen Fan cb06209fc9 [SPARK-28747][SQL] merge the two data source v2 fallback configs
## What changes were proposed in this pull request?

Currently we have 2 configs to specify which v2 sources should fallback to v1 code path. One config for read path, and one config for write path.

However, I found it's awkward to work with these 2 configs:
1. for `CREATE TABLE USING format`, should this be read path or write path?
2. for `V2SessionCatalog.loadTable`,  we need to return `UnresolvedTable` if it's a DS v1 or we need to fallback to v1 code path. However, at that time, we don't know if the returned table will be used for read or write.

We don't have any new features or perf improvement in file source v2. The fallback API is just a safeguard if we have bugs in v2 implementations. There are not many benefits to support falling back to v1 for read and write path separately.

This PR proposes to merge these 2 configs into one.

## How was this patch tested?

existing tests

Closes #25465 from cloud-fan/merge-conf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 20:47:24 +08:00
Yuming Wang ab1819d38a [SPARK-28527][SQL][TEST][FOLLOW-UP] Ignores Thrift server ThriftServerQueryTestSuite
### What changes were proposed in this pull request?

This PR ignores Thrift server `ThriftServerQueryTestSuite`.

### Why are the changes needed?

This ThriftServerQueryTestSuite test case led to frequent Jenkins build failure.

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

Yes.

### How was this patch tested?
N/A

Closes #25592 from wangyum/SPARK-28527-f1.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-27 15:41:22 +09:00
Burak Yavuz e31aec9be4 [SPARK-28667][SQL] Support InsertInto through the V2SessionCatalog
### What changes were proposed in this pull request?

This PR adds support for INSERT INTO through both the SQL and DataFrameWriter APIs through the V2SessionCatalog.

### Why are the changes needed?

This will allow V2 tables to be plugged in through the V2SessionCatalog, and be used seamlessly with existing APIs.

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

No behavior changes.

### How was this patch tested?

Pulled out a lot of tests so that they can be shared across the DataFrameWriter and SQL code paths.

Closes #25507 from brkyvz/insertSesh.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-27 12:59:53 +08:00
Yuming Wang 6e12b585a9 [SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server
### What changes were proposed in this pull request?
This PR build a test framework that directly re-run all the tests in `SQLQueryTestSuite` via Thrift Server. But it's a little different from `SQLQueryTestSuite`:
1. Can not support [UDF testing](44e607e921/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L293-L297)).
2. Can not support `DESC` command and `SHOW` command because `SQLQueryTestSuite` [formatted the output](1882912cca/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala (L38-L50).).

When building this framework, found two bug:
[SPARK-28624](https://issues.apache.org/jira/browse/SPARK-28624): `make_date` is inconsistent when reading from table
[SPARK-28611](https://issues.apache.org/jira/browse/SPARK-28611): Histogram's height is different

found two features that ThriftServer can not support:
[SPARK-28636](https://issues.apache.org/jira/browse/SPARK-28636): ThriftServer can not support decimal type with negative scale
[SPARK-28637](https://issues.apache.org/jira/browse/SPARK-28637): ThriftServer can not support interval type

Also, found two inconsistent behavior:
[SPARK-28620](https://issues.apache.org/jira/browse/SPARK-28620): Double type returned for float type in Beeline/JDBC
[SPARK-28619](https://issues.apache.org/jira/browse/SPARK-28619):  The golden result file is different when tested by `bin/spark-sql`

### Why are the changes needed?

Improve the overall test coverage for Thrift Server.

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

### How was this patch tested?
N/A

Closes #25567 from wangyum/SPARK-28527.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-26 22:39:57 +09:00
Dilip Biswal c61270fd74 [SPARK-27395][SQL] Improve EXPLAIN command
## What changes were proposed in this pull request?
This PR aims at improving the way physical plans are explained in spark.

Currently, the explain output for physical plan may look very cluttered and each operator's
string representation can be very wide and wraps around in the display making it little
hard to follow. This especially happens when explaining a query 1) Operating on wide tables
2) Has complex expressions etc.

This PR attempts to split the output into two sections. In the header section, we display
the basic operator tree with a number associated with each operator. In this section, we strictly
control what we output for each operator. In the footer section, each operator is verbosely
displayed. Based on the feedback from Maryann, the uncorrelated subqueries (SubqueryExecs) are not included in the main plan. They are printed separately after the main plan and can be
correlated by the originating expression id from its parent plan.

To illustrate, here is a simple plan displayed in old vs new way.

Example query1 :
```
EXPLAIN SELECT key, Max(val) FROM explain_temp1 WHERE key > 0 GROUP BY key HAVING max(val) > 0
```

Old :
```
*(2) Project [key#2, max(val)#15]
+- *(2) Filter (isnotnull(max(val#3)#18) AND (max(val#3)#18 > 0))
   +- *(2) HashAggregate(keys=[key#2], functions=[max(val#3)], output=[key#2, max(val)#15, max(val#3)#18])
      +- Exchange hashpartitioning(key#2, 200)
         +- *(1) HashAggregate(keys=[key#2], functions=[partial_max(val#3)], output=[key#2, max#21])
            +- *(1) Project [key#2, val#3]
               +- *(1) Filter (isnotnull(key#2) AND (key#2 > 0))
                  +- *(1) FileScan parquet default.explain_temp1[key#2,val#3] Batched: true, DataFilters: [isnotnull(key#2), (key#2 > 0)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(key), GreaterThan(key,0)], ReadSchema: struct<key:int,val:int>
```
New :
```
Project (8)
+- Filter (7)
   +- HashAggregate (6)
      +- Exchange (5)
         +- HashAggregate (4)
            +- Project (3)
               +- Filter (2)
                  +- Scan parquet default.explain_temp1 (1)

(1) Scan parquet default.explain_temp1 [codegen id : 1]
Output: [key#2, val#3]

(2) Filter [codegen id : 1]
Input     : [key#2, val#3]
Condition : (isnotnull(key#2) AND (key#2 > 0))

(3) Project [codegen id : 1]
Output    : [key#2, val#3]
Input     : [key#2, val#3]

(4) HashAggregate [codegen id : 1]
Input: [key#2, val#3]

(5) Exchange
Input: [key#2, max#11]

(6) HashAggregate [codegen id : 2]
Input: [key#2, max#11]

(7) Filter [codegen id : 2]
Input     : [key#2, max(val)#5, max(val#3)#8]
Condition : (isnotnull(max(val#3)#8) AND (max(val#3)#8 > 0))

(8) Project [codegen id : 2]
Output    : [key#2, max(val)#5]
Input     : [key#2, max(val)#5, max(val#3)#8]
```

Example Query2 (subquery):
```
SELECT * FROM   explain_temp1 WHERE  KEY = (SELECT Max(KEY) FROM   explain_temp2 WHERE  KEY = (SELECT Max(KEY) FROM   explain_temp3 WHERE  val > 0) AND val = 2) AND val > 3
```
Old:
```
*(1) Project [key#2, val#3]
+- *(1) Filter (((isnotnull(KEY#2) AND isnotnull(val#3)) AND (KEY#2 = Subquery scalar-subquery#39)) AND (val#3 > 3))
   :  +- Subquery scalar-subquery#39
   :     +- *(2) HashAggregate(keys=[], functions=[max(KEY#26)], output=[max(KEY)#45])
   :        +- Exchange SinglePartition
   :           +- *(1) HashAggregate(keys=[], functions=[partial_max(KEY#26)], output=[max#47])
   :              +- *(1) Project [key#26]
   :                 +- *(1) Filter (((isnotnull(KEY#26) AND isnotnull(val#27)) AND (KEY#26 = Subquery scalar-subquery#38)) AND (val#27 = 2))
   :                    :  +- Subquery scalar-subquery#38
   :                    :     +- *(2) HashAggregate(keys=[], functions=[max(KEY#28)], output=[max(KEY)#43])
   :                    :        +- Exchange SinglePartition
   :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_max(KEY#28)], output=[max#49])
   :                    :              +- *(1) Project [key#28]
   :                    :                 +- *(1) Filter (isnotnull(val#29) AND (val#29 > 0))
   :                    :                    +- *(1) FileScan parquet default.explain_temp3[key#28,val#29] Batched: true, DataFilters: [isnotnull(val#29), (val#29 > 0)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp3], PartitionFilters: [], PushedFilters: [IsNotNull(val), GreaterThan(val,0)], ReadSchema: struct<key:int,val:int>
   :                    +- *(1) FileScan parquet default.explain_temp2[key#26,val#27] Batched: true, DataFilters: [isnotnull(key#26), isnotnull(val#27), (val#27 = 2)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp2], PartitionFilters: [], PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(val,2)], ReadSchema: struct<key:int,val:int>
   +- *(1) FileScan parquet default.explain_temp1[key#2,val#3] Batched: true, DataFilters: [isnotnull(key#2), isnotnull(val#3), (val#3 > 3)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(key), IsNotNull(val), GreaterThan(val,3)], ReadSchema: struct<key:int,val:int>
```
New:
```
Project (3)
+- Filter (2)
   +- Scan parquet default.explain_temp1 (1)

(1) Scan parquet default.explain_temp1 [codegen id : 1]
Output: [key#2, val#3]

(2) Filter [codegen id : 1]
Input     : [key#2, val#3]
Condition : (((isnotnull(KEY#2) AND isnotnull(val#3)) AND (KEY#2 = Subquery scalar-subquery#23)) AND (val#3 > 3))

(3) Project [codegen id : 1]
Output    : [key#2, val#3]
Input     : [key#2, val#3]
===== Subqueries =====

Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery scalar-subquery#23
HashAggregate (9)
+- Exchange (8)
   +- HashAggregate (7)
      +- Project (6)
         +- Filter (5)
            +- Scan parquet default.explain_temp2 (4)

(4) Scan parquet default.explain_temp2 [codegen id : 1]
Output: [key#26, val#27]

(5) Filter [codegen id : 1]
Input     : [key#26, val#27]
Condition : (((isnotnull(KEY#26) AND isnotnull(val#27)) AND (KEY#26 = Subquery scalar-subquery#22)) AND (val#27 = 2))

(6) Project [codegen id : 1]
Output    : [key#26]
Input     : [key#26, val#27]

(7) HashAggregate [codegen id : 1]
Input: [key#26]

(8) Exchange
Input: [max#35]

(9) HashAggregate [codegen id : 2]
Input: [max#35]

Subquery:2 Hosting operator id = 5 Hosting Expression = Subquery scalar-subquery#22
HashAggregate (15)
+- Exchange (14)
   +- HashAggregate (13)
      +- Project (12)
         +- Filter (11)
            +- Scan parquet default.explain_temp3 (10)

(10) Scan parquet default.explain_temp3 [codegen id : 1]
Output: [key#28, val#29]

(11) Filter [codegen id : 1]
Input     : [key#28, val#29]
Condition : (isnotnull(val#29) AND (val#29 > 0))

(12) Project [codegen id : 1]
Output    : [key#28]
Input     : [key#28, val#29]

(13) HashAggregate [codegen id : 1]
Input: [key#28]

(14) Exchange
Input: [max#37]

(15) HashAggregate [codegen id : 2]
Input: [max#37]
```

Note:
I opened this PR as a WIP to start getting feedback. I will be on vacation starting tomorrow
would not be able to immediately incorporate the feedback. I will start to
work on them as soon as i can. Also, currently this PR provides a basic infrastructure
for explain enhancement. The details about individual operators will be implemented
in follow-up prs
## How was this patch tested?
Added a new test `explain.sql` that tests basic scenarios. Need to add more tests.

Closes #24759 from dilipbiswal/explain_feature.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-26 20:37:13 +08:00
Yuming Wang c353a84d1a [SPARK-28642][SQL][TEST][FOLLOW-UP] Test spark.sql.redaction.options.regex with and without default values
### What changes were proposed in this pull request?

Test `spark.sql.redaction.options.regex` with and without  default values.

### Why are the changes needed?

Normally, we do not rely on the default value of `spark.sql.redaction.options.regex`.

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

### How was this patch tested?
N/A

Closes #25579 from wangyum/SPARK-28642-f1.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-08-25 23:12:16 -07:00
Yuming Wang adb506afd7 [SPARK-28852][SQL] Implement SparkGetCatalogsOperation for Thrift Server
### What changes were proposed in this pull request?
This PR implements `SparkGetCatalogsOperation` for Thrift Server metadata completeness.

### Why are the changes needed?
Thrift Server metadata completeness.

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

### How was this patch tested?
Unit test

Closes #25555 from wangyum/SPARK-28852.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-08-25 22:42:50 -07:00
Terry Kim a3328cdc0a [SPARK-28238][SQL][FOLLOW-UP] Clean up attributes for Datasource v2 DESCRIBE TABLE
### What changes were proposed in this pull request?
1. Fix the physical plan (`DescribeTableExec`) to have the same output attributes as the corresponding logical plan.
2. Remove `output` in statements since they are unresolved plans.

### Why are the changes needed?
Correctness of how output attributes should work.

### Does this PR introduce any user-facing change?
NO

### How was this patch tested?
Existing tests

Closes #25568 from imback82/describe_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-26 13:39:36 +08:00
Yuming Wang 4b16cf11b3 [SPARK-27988][SQL][TEST] Port AGGREGATES.sql [Part 3]
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/aggregates.out#L986-L1613

When porting the test cases, found seven PostgreSQL specific features that do not exist in Spark SQL:

[SPARK-27974](https://issues.apache.org/jira/browse/SPARK-27974): Add built-in Aggregate Function: array_agg
[SPARK-27978](https://issues.apache.org/jira/browse/SPARK-27978): Add built-in Aggregate Functions: string_agg
[SPARK-27986](https://issues.apache.org/jira/browse/SPARK-27986): Support Aggregate Expressions with filter
[SPARK-27987](https://issues.apache.org/jira/browse/SPARK-27987): Support POSIX Regular Expressions
[SPARK-28682](https://issues.apache.org/jira/browse/SPARK-28682): ANSI SQL: Collation Support
[SPARK-28768](https://issues.apache.org/jira/browse/SPARK-28768): Implement more text pattern operators
[SPARK-28865](https://issues.apache.org/jira/browse/SPARK-28865): Table inheritance

## How was this patch tested?

N/A

Closes #24829 from wangyum/SPARK-27988.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-25 23:34:59 +09:00
Yuming Wang 02a0cdea13 [SPARK-28723][SQL] Upgrade to Hive 2.3.6 for HiveMetastore Client and Hadoop-3.2 profile
### What changes were proposed in this pull request?

This PR upgrade the built-in Hive to 2.3.6 for `hadoop-3.2`.

Hive 2.3.6 release notes:
- [HIVE-22096](https://issues.apache.org/jira/browse/HIVE-22096): Backport [HIVE-21584](https://issues.apache.org/jira/browse/HIVE-21584) (Java 11 preparation: system class loader is not URLClassLoader)
- [HIVE-21859](https://issues.apache.org/jira/browse/HIVE-21859): Backport [HIVE-17466](https://issues.apache.org/jira/browse/HIVE-17466) (Metastore API to list unique partition-key-value combinations)
- [HIVE-21786](https://issues.apache.org/jira/browse/HIVE-21786): Update repo URLs in poms branch 2.3 version

### Why are the changes needed?
Make Spark support JDK 11.

### Does this PR introduce any user-facing change?
Yes. Please see [SPARK-28684](https://issues.apache.org/jira/browse/SPARK-28684) and [SPARK-24417](https://issues.apache.org/jira/browse/SPARK-24417) for more details.

### How was this patch tested?
Existing unit test and manual test.

Closes #25443 from wangyum/test-on-jenkins.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-23 21:34:30 -07:00
Xiao Li 07c4b9bd1f Revert "[SPARK-25474][SQL] Support spark.sql.statistics.fallBackToHdfs in data source tables"
This reverts commit 485ae6d181.

Closes #25563 from gatorsmile/revert.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-23 07:41:39 -07:00
Gengliang Wang 8258660f67 [SPARK-28741][SQL] Optional mode: throw exceptions when casting to integers causes overflow
## What changes were proposed in this pull request?

To follow ANSI SQL, we should support a configurable mode that throws exceptions when casting to integers causes overflow.
The behavior is similar to https://issues.apache.org/jira/browse/SPARK-26218, which throws exceptions on arithmetical operation overflow.
To unify it, the configuration is renamed from "spark.sql.arithmeticOperations.failOnOverFlow" to "spark.sql.failOnIntegerOverFlow"
## How was this patch tested?

Unit test

Closes #25461 from gengliangwang/AnsiCastIntegral.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-23 21:49:45 +08:00
Ali Afroozeh 1472e664ba [SPARK-28716][SQL] Add id to Exchange and Subquery's stringArgs method for easier identifying their reuses in query plans
## What changes were proposed in this pull request?

Add id to Exchange and Subquery's stringArgs method for easier identifying their reuses in query plans, for example:
```
ReusedExchange d_date_sk#827, BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))) [id=#2710]
```
Where `2710` is the id of the reused exchange.

## How was this patch tested?

Passes existing tests

Closes #25434 from dbaliafroozeh/ImplementStringArgsExchangeSubqueryExec.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-08-23 13:29:32 +02:00
Ali Afroozeh aef7ca1f0b [SPARK-28836][SQL] Remove the canonicalize(attributes) method from PlanExpression
### What changes were proposed in this pull request?
This PR removes the `canonicalize(attrs: AttributeSeq)` from `PlanExpression` and taking care of normalizing expressions in `QueryPlan`.

### Why are the changes needed?
`Expression` has already a `canonicalized` method and having the `canonicalize` method in `PlanExpression` is confusing.

### Does this PR introduce any user-facing change?
Removes the `canonicalize` plan from `PlanExpression`. Also renames the `normalizeExprId` to `normalizeExpressions` in query plan.

### How was this patch tested?
This PR is a refactoring and passes the existing tests

Closes #25534 from dbaliafroozeh/ImproveCanonicalizeAPI.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-08-23 13:26:58 +02:00
terryk 98e1a4cea4 [SPARK-28319][SQL] Implement SHOW TABLES for Data Source V2 Tables
## What changes were proposed in this pull request?

Implements the SHOW TABLES logical and physical plans for data source v2 tables.

## How was this patch tested?

Added unit tests to `DataSourceV2SQLSuite`.

Closes #25247 from imback82/dsv2_show_tables.

Lead-authored-by: terryk <yuminkim@gmail.com>
Co-authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-23 14:20:25 +08:00
Ali Afroozeh 9976b876f1 [SPARK-28835][SQL][TEST] Add TPCDSSchema trait
### What changes were proposed in this pull request?
This PR extracts the schema information of TPCDS tables into a separate class called `TPCDSSchema` which can be reused for other testing purposes

### How was this patch tested?
This PR is only a refactoring for tests and passes existing tests

Closes #25535 from dbaliafroozeh/IntroduceTPCDSSchema.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-22 23:18:46 -07:00
Jungtaek Lim (HeartSaVioR) 406c5331ff
[SPARK-28025][SS] Fix FileContextBasedCheckpointFileManager leaking crc files
### What changes were proposed in this pull request?

This PR fixes the leak of crc files from CheckpointFileManager when FileContextBasedCheckpointFileManager is being used.

Spark hits the Hadoop bug, [HADOOP-16255](https://issues.apache.org/jira/browse/HADOOP-16255) which seems to be a long-standing issue.

This is there're two `renameInternal` methods:

```
public void renameInternal(Path src, Path dst)
public void renameInternal(final Path src, final Path dst, boolean overwrite)
```

which should be overridden to handle all cases but ChecksumFs only overrides method with 2 params, so when latter is called FilterFs.renameInternal(...) is called instead, and it will do rename with RawLocalFs as underlying filesystem.

The bug is related to FileContext, so FileSystemBasedCheckpointFileManager is not affected.

[SPARK-17475](https://issues.apache.org/jira/browse/SPARK-17475) took a workaround for this bug, but [SPARK-23966](https://issues.apache.org/jira/browse/SPARK-23966) seemed to bring regression.

This PR deletes crc file as "best-effort" when renaming, as failing to delete crc file is not that critical to fail the task.

### Why are the changes needed?

This PR prevents crc files not being cleaned up even purging batches. Too many files in same directory often hurts performance, as well as each crc file occupies more space than its own size so possible to occupy nontrivial amount of space when batches go up to 100000+.

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

No.

### How was this patch tested?

Some unit tests are modified to check leakage of crc files.

Closes #25488 from HeartSaVioR/SPARK-28025.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-08-22 23:10:16 -07:00
Gengliang Wang 895c90b582 [SPARK-28730][SQL] Configurable type coercion policy for table insertion
## What changes were proposed in this pull request?

After all the discussions in the dev list: http://apache-spark-developers-list.1001551.n3.nabble.com/Discuss-Follow-ANSI-SQL-on-table-insertion-td27531.html#a27562.
Here I propose that we can make the store assignment rules in the analyzer configurable, and the behavior of V1 and V2 should be consistent.
When inserting a value into a column with a different data type, Spark will perform type coercion. After this PR, we support 2 policies for the type coercion rules:
legacy and strict.
1. With legacy policy, Spark allows casting any value to any data type. The legacy policy is the only behavior in Spark 2.x and it is compatible with Hive.
2. With strict policy, Spark doesn't allow any possible precision loss or data truncation in type coercion, e.g. `int` and `long`, `float` -> `double` are not allowed.

Eventually, the "legacy" mode will be removed, so it is disallowed in data source V2.
To ensure backward compatibility with existing queries, the default store assignment policy for data source V1 is "legacy".
## How was this patch tested?

Unit test

Closes #25453 from gengliangwang/tableInsertRule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-23 13:50:26 +08:00
shivusondur 23bed0d3c0 [SPARK-28702][SQL] Display useful error message (instead of NPE) for invalid Dataset operations
### What changes were proposed in this pull request?
Added proper message instead of NPE for invalid Dataset operations (e.g. calling actions inside of transformations) similar to SPARK-5063 for RDD

### Why are the changes needed?
To report the user about the exact issue instead of NPE

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

### How was this patch tested?

Manually tested

```scala
test code snap
"import spark.implicits._
    val ds1 = spark.sparkContext.parallelize(1 to 100, 100).toDS()
    val ds2 = spark.sparkContext.parallelize(1 to 100, 100).toDS()
    ds1.map(x => {
      // scalastyle:off
      println(ds2.count + x)
      x
    }).collect()"
```

Closes #25503 from shivusondur/jira28702.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-08-22 22:15:37 -07:00
Dongjoon Hyun 36da2e3384 [SPARK-28847][TEST] Annotate HiveExternalCatalogVersionsSuite with ExtendedHiveTest
### What changes were proposed in this pull request?

This PR aims to annotate `HiveExternalCatalogVersionsSuite` with `ExtendedHiveTest`.

### Why are the changes needed?

`HiveExternalCatalogVersionsSuite` is an outstanding test in terms of testing time. This PR aims to allow skipping this test suite when we use `ExtendedHiveTest`.
![time](https://user-images.githubusercontent.com/9700541/63489184-4c75af00-c466-11e9-9e12-d250d4a23292.png)

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

No

### How was this patch tested?

Since Jenkins doesn't exclude `ExtendedHiveTest`, there is no difference in Jenkins testing.
This PR should be tested by manually by the following.

**BEFORE**
```
$ cd sql/hive
$ mvn package -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest
...
Run starting. Expected test count is: 1
HiveExternalCatalogVersionsSuite:
22:32:16.218 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load ...
```

**AFTER**
```
$ cd sql/hive
$ mvn package -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest
...
Run starting. Expected test count is: 0
HiveExternalCatalogVersionsSuite:
Run completed in 772 milliseconds.
Total number of tests run: 0
Suites: completed 2, aborted 0
Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0
No tests were executed.
...
```

Closes #25550 from dongjoon-hyun/SPARK-28847.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-22 00:25:56 -07:00
triplesheep 48578a41b5 [SPARK-28844][SQL] Fix typo in SQLConf FILE_COMRESSION_FACTOR
### What changes were proposed in this pull request?
Fix minor typo in SQLConf.
`FILE_COMRESSION_FACTOR` -> `FILE_COMPRESSION_FACTOR`

### Why are the changes needed?
Make conf more understandable.

### Does this PR introduce any user-facing change?
No. (`spark.sql.sources.fileCompressionFactor` is unchanged.)

### How was this patch tested?
Pass the Jenkins with the existing tests.

Closes #25538 from triplesheep/TYPO-FIX.

Authored-by: triplesheep <triplesheep0419@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-22 00:07:40 -07:00
maryannxue aefb2e70e7 [SPARK-28739][SQL] Add a simple cost check for Adaptive Query Execution
### What changes were proposed in this pull request?

This PR adds a simple cost model and a mechanism to compare the costs of the before and after plans of each re-optimization in Adaptive Query Execution. Now the workflow of AQE re-optimization is changed to: If the cost of the plan after re-optimization is lower than or equal to that of the plan before re-optimization and the plan has been changed after re-optimization (if equal), the current physical plan will be updated to the plan after re-optimization, otherwise it will remain unchanged until the next re-optimization.

### Why are the changes needed?
This new mechanism is to prevent regressions in Adaptive Query Execution caused by change of the plan introducing extra cost, in this PR specifically, change of SMJ to BHJ leading to extra `ShuffleExchangeExec`s.

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

### How was this patch tested?
Added UT.

Closes #25456 from maryannxue/aqe-cost.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-21 19:33:56 -07:00
Wenchen Fan ed3ea6734c [SPARK-28837][SQL] CTAS/RTAS should use nullable schema
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
When running CTAS/RTAS, use the nullable schema of the input query to create the table.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
It's very likely to run CTAS/RTAS with non-nullable input query, e.g. `CREATE TABLE t AS SELECT 1`. However, it's surprising to users if they can't write null to this table later. Non-nullable is kind of a constraint of the column and should be specified by users explicitly.

For reference, Postgres also use nullable schema for CTAS:
```
> create table t1(i int not null);

> insert into t1 values (1);

> create table t2 as select i from t1;

> \d+ t1;
 Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
 i      | integer |           | not null |         | plain   |              |

> \d+ t2;
 Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
 i      | integer |           |          |         | plain   |              |

```

File source V1 has the same behavior.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
Yes, after this PR CTAS/RTAS creates tables with nullable schema, then users can insert null values later.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
new test

Closes #25536 from cloud-fan/ctas.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-22 09:49:18 +08:00
Wenchen Fan 97b046f06f [SPARK-28635][SQL][FOLLOWUP] CatalogManager should reflect the changes of default catalog
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
The current namespace/catalog should be set to None at the beginning, so that we can read the new configs when reporting currennt namespace/catalog later.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
Fix a bug in CatalogManager, to reflect the change of default catalog config when reporting current catalog.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
No. The current namespace/catalog stuff is still internal right now.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
a new test suite

Closes #25521 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-21 12:23:42 -07:00
Yuanjian Li 2d9cc42aa8 [SPARK-28699][SQL] Disable using radix sort for ShuffleExchangeExec in repartition case
## What changes were proposed in this pull request?

Disable using radix sort in ShuffleExchangeExec when we do repartition.
In #20393, we fixed the indeterministic result in the shuffle repartition case by performing a local sort before repartitioning.
But for the newly added sort operation, we use radix sort which is wrong because binary data can't be compared by only the prefix. This makes the sort unstable and fails to solve the indeterminate shuffle output problem.

### Why are the changes needed?
Fix the correctness bug caused by repartition after a shuffle.

### Does this PR introduce any user-facing change?
Yes, user will get the right result in the case of repartition stage rerun.

## How was this patch tested?

Test with `local-cluster[5, 2, 5120]`, use the integrated test below, it can return a right answer 100000000.
```
import scala.sys.process._
import org.apache.spark.TaskContext

val res = spark.range(0, 10000 * 10000, 1).map{ x => (x % 1000, x)}
// kill an executor in the stage that performs repartition(239)
val df = res.repartition(113).map{ x => (x._1 + 1, x._2)}.repartition(239).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 1 && TaskContext.get.stageAttemptNumber == 0) {
    throw new Exception("pkill -f -n java".!!)
  }
  x
}
val r2 = df.distinct.count()
```

Closes #25491 from xuanyuanking/SPARK-28699-fix.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-21 10:56:50 -07:00
Ali Afroozeh 4dc3093513 [SPARK-28715][SQL] Introduce collectInPlanAndSubqueries and subqueriesAll in QueryPlan
## What changes were proposed in this pull request?

Introduces the collectInPlanAndSubqueries and subqueriesAll methods in QueryPlan that consider all the plans in the query plan, including the ones in nested subqueries.

## How was this patch tested?

Unit test added

Closes #25433 from dbaliafroozeh/IntroduceCollectInPlanAndSubqueries.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-08-21 18:05:18 +02:00
Robert (Bobby) Evans fac469e2e0 [SPARK-28774][SQL] Fix exchange reuse for columnar data
### What changes were proposed in this pull request?
The rule ReuseExchange optimization rule will look for instances of Exchange that have the same plan and convert dedupe them to them to a ReuseExchangeExec instance. In the current Spark codebase all Exchange instances are row based, but if we use the spark.sql.extensions config to put in our own columnar based exchange implementation reuse will throw an exception saying that there was a columnar mismatch.

### Why are the changes needed?
Without it Reused Columnar Exchanges throw an exception

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

### How was this patch tested?

I tested this patch by running it against a query that was showing this exact issue and it fixed it.

I also added a very simple unit test that shows the issue.

Closes #25499 from revans2/reused-columnar-exchange.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-21 18:10:26 +08:00
Burak Yavuz 4855bfe16b [SPARK-28554][SQL] Adds a v1 fallback writer implementation for v2 data source codepaths
## What changes were proposed in this pull request?

This PR adds a V1 fallback interface for writing to V2 Tables using V1 Writer interfaces. The only supported SaveMode that will be called on the target table will be an Append. The target table must use V2 interfaces such as `SupportsOverwrite` or `SupportsTruncate` to support Overwrite operations. It is up to the target DataSource implementation if this operation can be atomic or not.

We do not support dynamicPartitionOverwrite, as we cannot call a `commit` method that actually cleans up the data in the partitions that were touched through this fallback.

## How was this patch tested?

Will add tests and example implementation after comments + feedback. This is a proposal at this point.

Closes #25348 from brkyvz/v1WriteFallback.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-21 17:25:25 +08:00
Marco Gaido 0bfcf9c210 [SPARK-28322][SQL] Add support to Decimal type for integral divide
## What changes were proposed in this pull request?

The expression `IntegralDivide`, which corresponds to the `div` operator, support only integral type. Postgres, though, allows it to work also with decimals.

The PR adds the support to decimal operands for this operation in order to have feature parity with postgres.

## How was this patch tested?

added UTs

Closes #25136 from mgaido91/SPARK-28322.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-21 08:43:00 +09:00
maryannxue 39c11273e0 [SPARK-28753][SQL] Dynamically reuse subqueries in AQE
### What changes were proposed in this pull request?
This PR changes subquery reuse in Adaptive Query Execution from compile-time static reuse to execution-time dynamic reuse. This PR adds a `ReuseAdaptiveSubquery` rule that applies to a query stage after it is created and before it is executed. The new dynamic reuse enables subqueries to be reused across all different subquery levels.

### Why are the changes needed?
This is an improvement to the current subquery reuse in Adaptive Query Execution, which allows subquery reuse to happen in a lazy fashion as well as at different subquery levels.

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

### How was this patch tested?
Passed existing tests.

Closes #25471 from maryannxue/aqe-dynamic-sub-reuse.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 19:58:29 +08:00
Wenchen Fan d04522187a [SPARK-28635][SQL] create CatalogManager to track registered v2 catalogs
## What changes were proposed in this pull request?

This is a pure refactor PR, which creates a new class `CatalogManager` to track the registered v2 catalogs, and provide the catalog up functionality.

`CatalogManager` also tracks the current catalog/namespace. We will implement corresponding commands in other PRs, like `USE CATALOG my_catalog`

## How was this patch tested?

existing tests

Closes #25368 from cloud-fan/refactor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 19:40:21 +08:00
Jungtaek Lim (HeartSaVioR) b37c8d5cea
[SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter
#  What changes were proposed in this pull request?

This patch modifies the explanation of guarantee for ForeachWriter as it doesn't guarantee same output for `(partitionId, epochId)`. Refer the description of [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for more details.

Spark itself still guarantees same output for same epochId (batch) if the preconditions are met, 1) source is always providing the same input records for same offset request. 2) the query is idempotent in overall (indeterministic calculation like now(), random() can break this).

Assuming breaking preconditions as an exceptional case (the preconditions are implicitly required even before), we still can describe the guarantee with `epochId`, though it will be  harder to leverage the guarantee: 1) ForeachWriter should implement a feature to track whether all the partitions are written successfully for given `epochId` 2) There's pretty less chance to leverage the fact, as the chance for Spark to successfully write all partitions and fail to checkpoint the batch is small.

Credit to zsxwing on discovering the broken guarantee.

## How was this patch tested?

This is just a documentation change, both on javadoc and guide doc.

Closes #25407 from HeartSaVioR/SPARK-28650.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-08-20 00:56:53 -07:00
lihao 79464bed2f [SPARK-28662][SQL] Create Hive Partitioned Table DDL should fail when partition column type missed
## What changes were proposed in this pull request?
Create Hive Partitioned Table without specifying data type for partition column will success unexpectedly.
```HiveQL
// create a hive table partition by b, but the data type of b isn't specified.
CREATE TABLE tbl(a int) PARTITIONED BY (b) STORED AS parquet
```
In https://issues.apache.org/jira/browse/SPARK-26435 ,  PARTITIONED BY clause  are extended to support Hive CTAS as following:
```ANTLR
// Before
(PARTITIONED BY '(' partitionColumns=colTypeList ')'

 // After
(PARTITIONED BY '(' partitionColumns=colTypeList ')'|
PARTITIONED BY partitionColumnNames=identifierList) |
```

Create Table Statement like above case will pass the syntax check,  and recognized as (PARTITIONED BY partitionColumnNames=identifierList) 。

This PR  will check this case in visitCreateHiveTable and throw a exception which contains  explicit error message to user.

## How was this patch tested?

Added tests.

Closes #25390 from lidinghao/hive-ddl-fix.

Authored-by: lihao <lihaowhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 14:37:04 +08:00
Sean Owen 3b4e345fa1 [SPARK-28775][CORE][TESTS] Skip date 8633 in Kwajalein due to changes in tzdata2018i that only some JDK 8s use
### What changes were proposed in this pull request?

Some newer JDKs use the tzdata2018i database, which changes how certain (obscure) historical dates and timezones are handled. As previously, we can pretty much safely ignore these in tests, as the value may vary by JDK.

### Why are the changes needed?

Test otherwise fails using, for example, JDK 1.8.0_222. https://bugs.openjdk.java.net/browse/JDK-8215982 has a full list of JDKs which has this.

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

No.

### How was this patch tested?

Existing tests

Closes #25504 from srowen/SPARK-28775.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-19 17:54:25 -07:00
Mick Jermsurawong b79cf0d143 [SPARK-28224][SQL] Check overflow in decimal Sum aggregate
## What changes were proposed in this pull request?
- Currently `sum` in aggregates for decimal type can overflow and return null.
  - `Sum` expression codegens arithmetic on `sql.Decimal` and the output which preserves scale and precision goes into `UnsafeRowWriter`. Here overflowing will be converted to null when writing out.
  - It also does not go through this branch in `DecimalAggregates` because it's expecting precision of the sum (not the elements to be summed) to be less than 5.
4ebff5b6d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala (L1400-L1403)

- This PR adds the check at the final result of the sum operator itself.
4ebff5b6d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala (L372-L376)

https://issues.apache.org/jira/browse/SPARK-28224

## How was this patch tested?

- Added an integration test on dataframe suite

cc mgaido91 JoshRosen

Closes #25033 from mickjermsurawong-stripe/SPARK-28224.

Authored-by: Mick Jermsurawong <mickjermsurawong@stripe.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-20 09:47:04 +09:00
Takuya UESHIN 26f344354b [SPARK-27905][SQL][FOLLOW-UP] Add prettyNames
### What changes were proposed in this pull request?

This is a follow-up of #24761 which added a higher-order function `ArrayForAll`.
The PR mistakenly removed the `prettyName` from `ArrayExists` and forgot to add it to `ArrayForAll`.

### Why are the changes needed?

This reverts the `prettyName` back to `ArrayExists` not to affect explained plans, and adds it to `ArrayForAll` to clarify the `prettyName` as the same as the expressions around.

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

No.

### How was this patch tested?

Existing tests.

Closes #25501 from ueshin/issues/SPARK-27905/pretty_names.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-19 15:15:50 -07:00
Huaxin Gao ec14b6eb65 [SPARK-28393][SQL][PYTHON][TESTS] Convert and port 'pgSQL/join.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from ```pgSQL/join.sql``` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).
<details><summary>Diff comparing to 'join.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out
index f75fe05196..ad2b5dd0db 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out
 -240,10 +240,10  struct<>

 -- !query 27
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t)
   FROM J1_TBL AS tx
 -- !query 27 schema
-struct<xxx:string,i:int,j:int,t:string>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string>
 -- !query 27 output
        0       NULL    zero
        1       4       one
 -259,10 +259,10  struct<xxx:string,i:int,j:int,t:string>

 -- !query 28
-SELECT '' AS `xxx`, *
+SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t)
   FROM J1_TBL tx
 -- !query 28 schema
-struct<xxx:string,i:int,j:int,t:string>
+struct<xxx:string,CAST(udf(cast(cast(udf(cast(i as string)) as int) as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string>
 -- !query 28 output
        0       NULL    zero
        1       4       one
 -278,10 +278,10  struct<xxx:string,i:int,j:int,t:string>

 -- !query 29
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, a, udf(udf(b)), c
   FROM J1_TBL AS t1 (a, b, c)
 -- !query 29 schema
-struct<xxx:string,a:int,b:int,c:string>
+struct<xxx:string,a:int,CAST(udf(cast(cast(udf(cast(b as string)) as int) as string)) AS INT):int,c:string>
 -- !query 29 output
        0       NULL    zero
        1       4       one
 -297,10 +297,10  struct<xxx:string,a:int,b:int,c:string>

 -- !query 30
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c))
   FROM J1_TBL t1 (a, b, c)
 -- !query 30 schema
-struct<xxx:string,a:int,b:int,c:string>
+struct<xxx:string,CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(c as string)) as string) as string)) AS STRING):string>
 -- !query 30 output
        0       NULL    zero
        1       4       one
 -316,10 +316,10  struct<xxx:string,a:int,b:int,c:string>

 -- !query 31
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e
   FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e)
 -- !query 31 schema
-struct<xxx:string,a:int,b:int,c:string,d:int,e:int>
+struct<xxx:string,CAST(udf(cast(a as string)) AS INT):int,b:int,CAST(udf(cast(c as string)) AS STRING):string,CAST(udf(cast(d as string)) AS INT):int,e:int>
 -- !query 31 output
        0       NULL    zero    0       NULL
        0       NULL    zero    1       -1
 -423,7 +423,7  struct<xxx:string,a:int,b:int,c:string,d:int,e:int>

 -- !query 32
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, *
   FROM J1_TBL CROSS JOIN J2_TBL
 -- !query 32 schema
 struct<xxx:string,i:int,j:int,t:string,i:int,k:int>
 -530,20 +530,20  struct<xxx:string,i:int,j:int,t:string,i:int,k:int>

 -- !query 33
-SELECT '' AS `xxx`, i, k, t
+SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t
   FROM J1_TBL CROSS JOIN J2_TBL
 -- !query 33 schema
 struct<>
 -- !query 33 output
 org.apache.spark.sql.AnalysisException
-Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 20
+Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 29

 -- !query 34
-SELECT '' AS `xxx`, t1.i, k, t
+SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t)
   FROM J1_TBL t1 CROSS JOIN J2_TBL t2
 -- !query 34 schema
-struct<xxx:string,i:int,k:int,t:string>
+struct<xxx:string,i:int,CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string>
 -- !query 34 output
        0       -1      zero
        0       -3      zero
 -647,11 +647,11  struct<xxx:string,i:int,k:int,t:string>

 -- !query 35
-SELECT '' AS `xxx`, ii, tt, kk
+SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk))
   FROM (J1_TBL CROSS JOIN J2_TBL)
     AS tx (ii, jj, tt, ii2, kk)
 -- !query 35 schema
-struct<xxx:string,ii:int,tt:string,kk:int>
+struct<xxx:string,ii:int,tt:string,CAST(udf(cast(cast(udf(cast(kk as string)) as int) as string)) AS INT):int>
 -- !query 35 output
        0       zero    -1
        0       zero    -3
 -755,10 +755,10  struct<xxx:string,ii:int,tt:string,kk:int>

 -- !query 36
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i),  udf(b.k)
   FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b
 -- !query 36 schema
-struct<xxx:string,i:int,j:int,t:string,i:int,k:int,i:int,k:int>
+struct<xxx:string,CAST(udf(cast(cast(udf(cast(i as string)) as int) as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 36 output
        0       NULL    zero    0       NULL    0       NULL
        0       NULL    zero    0       NULL    1       -1
 -1654,10 +1654,10  struct<xxx:string,i:int,j:int,t:string,i:int,k:int,i:int,k:int>

 -- !query 37
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k)
   FROM J1_TBL INNER JOIN J2_TBL USING (i)
 -- !query 37 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,i:int,CAST(udf(cast(j as string)) AS INT):int,t:string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 37 output
        0       NULL    zero    NULL
        1       4       one     -1
 -1669,10 +1669,10  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 38
-SELECT '' AS `xxx`, *
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k
   FROM J1_TBL JOIN J2_TBL USING (i)
 -- !query 38 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,j:int,CAST(udf(cast(t as string)) AS STRING):string,k:int>
 -- !query 38 output
        0       NULL    zero    NULL
        1       4       one     -1
 -1684,9 +1684,9  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 39
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, *
   FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a)
-  ORDER BY a, d
+  ORDER BY udf(udf(a)), udf(d)
 -- !query 39 schema
 struct<xxx:string,a:int,b:int,c:string,d:int>
 -- !query 39 output
 -1700,10 +1700,10  struct<xxx:string,a:int,b:int,c:string,d:int>

 -- !query 40
-SELECT '' AS `xxx`, *
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k)
   FROM J1_TBL NATURAL JOIN J2_TBL
 -- !query 40 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 40 output
        0       NULL    zero    NULL
        1       4       one     -1
 -1715,10 +1715,10  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 41
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d)
   FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d)
 -- !query 41 schema
-struct<xxx:string,a:int,b:int,c:string,d:int>
+struct<xxx:string,a:int,CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(c as string)) AS STRING):string,CAST(udf(cast(d as string)) AS INT):int>
 -- !query 41 output
        0       NULL    zero    NULL
        1       4       one     -1
 -1730,10 +1730,10  struct<xxx:string,a:int,b:int,c:string,d:int>

 -- !query 42
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d
   FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a)
 -- !query 42 schema
-struct<xxx:string,a:int,b:int,c:string,d:int>
+struct<xxx:string,CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(b as string)) as int) as string)) AS INT):int,c:string,d:int>
 -- !query 42 output
        0       NULL    zero    NULL
        2       3       two     2
 -1741,10 +1741,10  struct<xxx:string,a:int,b:int,c:string,d:int>

 -- !query 43
-SELECT '' AS `xxx`, *
-  FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.i)
+SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k)
+  FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i)
 -- !query 43 schema
-struct<xxx:string,i:int,j:int,t:string,i:int,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(j as string)) as int) as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 43 output
        0       NULL    zero    0       NULL
        1       4       one     1       -1
 -1756,10 +1756,10  struct<xxx:string,i:int,j:int,t:string,i:int,k:int>

 -- !query 44
-SELECT '' AS `xxx`, *
-  FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.k)
+SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k
+  FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k))
 -- !query 44 schema
-struct<xxx:string,i:int,j:int,t:string,i:int,k:int>
+struct<xxx:string,CAST(udf(cast(cast(udf(cast(i as string)) as int) as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(j as string)) as int) as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(t as string)) as string) as string)) AS STRING):string,i:int,k:int>
 -- !query 44 output
        0       NULL    zero    NULL    0
        2       3       two     2       2
 -1767,10 +1767,10  struct<xxx:string,i:int,j:int,t:string,i:int,k:int>

 -- !query 45
-SELECT '' AS `xxx`, *
-  FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i <= J2_TBL.k)
+SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k)
+  FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k)))
 -- !query 45 schema
-struct<xxx:string,i:int,j:int,t:string,i:int,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 45 output
        0       NULL    zero    2       2
        0       NULL    zero    2       4
 -1784,11 +1784,11  struct<xxx:string,i:int,j:int,t:string,i:int,k:int>

 -- !query 46
-SELECT '' AS `xxx`, *
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k)
   FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i)
-  ORDER BY i, k, t
+  ORDER BY udf(udf(i)), udf(k), udf(t)
 -- !query 46 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 46 output
        NULL    NULL    null    NULL
        NULL    0       zero    NULL
 -1806,11 +1806,11  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 47
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k)
   FROM J1_TBL LEFT JOIN J2_TBL USING (i)
-  ORDER BY i, k, t
+  ORDER BY udf(i), udf(udf(k)), udf(t)
 -- !query 47 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 47 output
        NULL    NULL    null    NULL
        NULL    0       zero    NULL
 -1828,10 +1828,10  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 48
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k)
   FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i)
 -- !query 48 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(cast(udf(cast(i as string)) as int) as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 48 output
        0       NULL    zero    NULL
        1       4       one     -1
 -1845,10 +1845,10  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 49
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k)
   FROM J1_TBL RIGHT JOIN J2_TBL USING (i)
 -- !query 49 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(j as string)) as int) as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 49 output
        0       NULL    zero    NULL
        1       4       one     -1
 -1862,11 +1862,11  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 50
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k)
   FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i)
-  ORDER BY i, k, t
+  ORDER BY udf(udf(i)), udf(k), udf(t)
 -- !query 50 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(t as string)) as string) as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 50 output
        NULL    NULL    NULL    NULL
        NULL    NULL    null    NULL
 -1886,11 +1886,11  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 51
-SELECT '' AS `xxx`, *
+SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k))
   FROM J1_TBL FULL JOIN J2_TBL USING (i)
-  ORDER BY i, k, t
+  ORDER BY udf(udf(i)), udf(k), udf(udf(t))
 -- !query 51 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,t:string,CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int>
 -- !query 51 output
        NULL    NULL    NULL    NULL
        NULL    NULL    null    NULL
 -1910,19 +1910,19  struct<xxx:string,i:int,j:int,t:string,k:int>

 -- !query 52
-SELECT '' AS `xxx`, *
-  FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (k = 1)
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k))
+  FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1)
 -- !query 52 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int>
 -- !query 52 output

 -- !query 53
-SELECT '' AS `xxx`, *
-  FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (i = 1)
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+  FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1))
 -- !query 53 schema
-struct<xxx:string,i:int,j:int,t:string,k:int>
+struct<xxx:string,CAST(udf(cast(i as string)) AS INT):int,CAST(udf(cast(j as string)) AS INT):int,CAST(udf(cast(t as string)) AS STRING):string,CAST(udf(cast(k as string)) AS INT):int>
 -- !query 53 output
        1       4       one     -1

 -2020,9 +2020,9  ee        NULL    42      NULL

 -- !query 65
 SELECT * FROM
-(SELECT * FROM t2) as s2
+(SELECT udf(name) as name, t2.n FROM t2) as s2
 INNER JOIN
-(SELECT * FROM t3) s3
+(SELECT udf(udf(name)) as name, t3.n FROM t3) s3
 USING (name)
 -- !query 65 schema
 struct<name:string,n:int,n:int>
 -2033,9 +2033,9  cc        22      23

 -- !query 66
 SELECT * FROM
-(SELECT * FROM t2) as s2
+(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2
 LEFT JOIN
-(SELECT * FROM t3) s3
+(SELECT udf(name) as name, t3.n FROM t3) s3
 USING (name)
 -- !query 66 schema
 struct<name:string,n:int,n:int>
 -2046,13 +2046,13  ee      42      NULL

 -- !query 67
-SELECT * FROM
+SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM
 (SELECT * FROM t2) as s2
 FULL JOIN
 (SELECT * FROM t3) s3
 USING (name)
 -- !query 67 schema
-struct<name:string,n:int,n:int>
+struct<CAST(udf(cast(name as string)) AS STRING):string,CAST(udf(cast(cast(udf(cast(n as string)) as int) as string)) AS INT):int,CAST(udf(cast(n as string)) AS INT):int>
 -- !query 67 output
 bb     12      13
 cc     22      23
 -2062,9 +2062,9  ee        42      NULL

 -- !query 68
 SELECT * FROM
-(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2
+(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2
 NATURAL INNER JOIN
-(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3
+(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3
 -- !query 68 schema
 struct<name:string,s2_n:int,s2_2:int,s3_n:int,s3_2:int>
 -- !query 68 output
 -2074,9 +2074,9  cc        22      2       23      3

 -- !query 69
 SELECT * FROM
-(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2
+(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2
 NATURAL LEFT JOIN
-(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3
+(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3
 -- !query 69 schema
 struct<name:string,s2_n:int,s2_2:int,s3_n:int,s3_2:int>
 -- !query 69 output
 -2087,9 +2087,9  ee        42      2       NULL    NULL

 -- !query 70
 SELECT * FROM
-(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2
+(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2
 NATURAL FULL JOIN
-(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3
+(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3
 -- !query 70 schema
 struct<name:string,s2_n:int,s2_2:int,s3_n:int,s3_2:int>
 -- !query 70 output
 -2101,11 +2101,11  ee      42      2       NULL    NULL

 -- !query 71
 SELECT * FROM
-(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1
+(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1
 NATURAL INNER JOIN
-(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2
+(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2
 NATURAL INNER JOIN
-(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3
+(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3
 -- !query 71 schema
 struct<name:string,s1_n:int,s1_1:int,s2_n:int,s2_2:int,s3_n:int,s3_2:int>
 -- !query 71 output
 -2114,11 +2114,11  bb      11      1       12      2       13      3

 -- !query 72
 SELECT * FROM
-(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1
+(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1
 NATURAL FULL JOIN
-(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2
+(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2
 NATURAL FULL JOIN
-(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3
+(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3
 -- !query 72 schema
 struct<name:string,s1_n:int,s1_1:int,s2_n:int,s2_2:int,s3_n:int,s3_2:int>
 -- !query 72 output
 -2129,16 +2129,16  ee      NULL    NULL    42      2       NULL    NULL

 -- !query 73
-SELECT * FROM
-(SELECT name, n as s1_n FROM t1) as s1
+SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM
+(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1
 NATURAL FULL JOIN
   (SELECT * FROM
-    (SELECT name, n as s2_n FROM t2) as s2
+    (SELECT name, udf(n) as s2_n FROM t2) as s2
     NATURAL FULL JOIN
-    (SELECT name, n as s3_n FROM t3) as s3
+    (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3
   ) ss2
 -- !query 73 schema
-struct<name:string,s1_n:int,s2_n:int,s3_n:int>
+struct<name:string,CAST(udf(cast(cast(udf(cast(s1_n as string)) as int) as string)) AS INT):int,CAST(udf(cast(s2_n as string)) AS INT):int,CAST(udf(cast(s3_n as string)) AS INT):int>
 -- !query 73 output
 bb     11      12      13
 cc     NULL    22      23
 -2151,9 +2151,9  SELECT * FROM
 (SELECT name, n as s1_n FROM t1) as s1
 NATURAL FULL JOIN
   (SELECT * FROM
-    (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2
+    (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2
     NATURAL FULL JOIN
-    (SELECT name, n as s3_n FROM t3) as s3
+    (SELECT name, udf(n) as s3_n FROM t3) as s3
   ) ss2
 -- !query 74 schema
 struct<name:string,s1_n:int,s2_n:int,s2_2:int,s3_n:int>
 -2165,13 +2165,13  ee      NULL    42      2       NULL

 -- !query 75
-SELECT * FROM
-  (SELECT name, n as s1_n FROM t1) as s1
+SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM
+  (SELECT name, udf(n) as s1_n FROM t1) as s1
 FULL JOIN
   (SELECT name, 2 as s2_n FROM t2) as s2
-ON (s1_n = s2_n)
+ON (udf(udf(s1_n)) = udf(s2_n))
 -- !query 75 schema
-struct<name:string,s1_n:int,name:string,s2_n:int>
+struct<name:string,CAST(udf(cast(s1_n as string)) AS INT):int,name:string,CAST(udf(cast(cast(udf(cast(s2_n as string)) as int) as string)) AS INT):int>
 -- !query 75 output
 NULL   NULL    bb      2
 NULL   NULL    cc      2
 -2200,9 +2200,9  struct<>

 -- !query 78
-select * from x
+select udf(udf(x1)), udf(x2) from x
 -- !query 78 schema
-struct<x1:int,x2:int>
+struct<CAST(udf(cast(cast(udf(cast(x1 as string)) as int) as string)) AS INT):int,CAST(udf(cast(x2 as string)) AS INT):int>
 -- !query 78 output
 1      11
 2      22
 -2212,9 +2212,9  struct<x1:int,x2:int>

 -- !query 79
-select * from y
+select udf(y1), udf(udf(y2)) from y
 -- !query 79 schema
-struct<y1:int,y2:int>
+struct<CAST(udf(cast(y1 as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(y2 as string)) as int) as string)) AS INT):int>
 -- !query 79 output
 1      111
 2      222
 -2223,7 +2223,7  struct<y1:int,y2:int>

 -- !query 80
-select * from x left join y on (x1 = y1 and x2 is not null)
+select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null)
 -- !query 80 schema
 struct<x1:int,x2:int,y1:int,y2:int>
 -- !query 80 output
 -2235,7 +2235,7  struct<x1:int,x2:int,y1:int,y2:int>

 -- !query 81
-select * from x left join y on (x1 = y1 and y2 is not null)
+select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null)
 -- !query 81 schema
 struct<x1:int,x2:int,y1:int,y2:int>
 -- !query 81 output
 -2247,8 +2247,8  struct<x1:int,x2:int,y1:int,y2:int>

 -- !query 82
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1)
+select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2)
+on (udf(udf(x1)) = udf(xx1))
 -- !query 82 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 82 output
 -2260,8 +2260,8  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 83
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1 and x2 is not null)
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = xx1 and udf(x2) is not null)
 -- !query 83 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 83 output
 -2273,8 +2273,8  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 84
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1 and y2 is not null)
+select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null)
 -- !query 84 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 84 output
 -2286,8 +2286,8  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 85
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1 and xx2 is not null)
+select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2)
+on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null)
 -- !query 85 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 85 output
 -2299,8 +2299,8  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 86
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1) where (x2 is not null)
+select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2)
+on (udf(x1) = udf(xx1)) where (udf(x2) is not null)
 -- !query 86 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 86 output
 -2310,8 +2310,8  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 87
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1) where (y2 is not null)
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = xx1) where (udf(y2) is not null)
 -- !query 87 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 87 output
 -2321,8 +2321,8  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 88
-select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2)
-on (x1 = xx1) where (xx2 is not null)
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (x1 = udf(xx1)) where (xx2 is not null)
 -- !query 88 schema
 struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>
 -- !query 88 output
 -2332,75 +2332,75  struct<x1:int,x2:int,y1:int,y2:int,xx1:int,xx2:int>

 -- !query 89
-select count(*) from tenk1 a where unique1 in
-  (select unique1 from tenk1 b join tenk1 c using (unique1)
-   where b.unique2 = 42)
+select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in
+  (select udf(unique1) from tenk1 b join tenk1 c using (unique1)
+   where udf(udf(b.unique2)) = udf(42))
 -- !query 89 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(cast(udf(cast(count(1) as string)) as bigint) as string)) AS BIGINT):bigint>
 -- !query 89 output
 1

 -- !query 90
-select count(*) from tenk1 x where
-  x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and
-  x.unique1 = 0 and
-  x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1)
+select udf(count(*)) from tenk1 x where
+  udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and
+  udf(x.unique1) = 0 and
+  udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1)))
 -- !query 90 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 90 output
 1

 -- !query 91
-select count(*) from tenk1 x where
-  x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and
-  x.unique1 = 0 and
-  x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1)
+select udf(udf(count(*))) from tenk1 x where
+  udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and
+  udf(x.unique1) = 0 and
+  udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1)))
 -- !query 91 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(cast(udf(cast(count(1) as string)) as bigint) as string)) AS BIGINT):bigint>
 -- !query 91 output
 1

 -- !query 92
 select * from int8_tbl i1 left join (int8_tbl i2 join
-  (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2
-order by 1, 2
+  (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2))
+order by udf(udf(1)), 2
 -- !query 92 schema
 struct<q1:bigint,q2:bigint,q1:bigint,q2:bigint,x:int>
 -- !query 92 output
-123    456     123     456     123
-123    4567890123456789        123     4567890123456789        123
 4567890123456789       -4567890123456789       NULL    NULL    NULL
 4567890123456789       123     NULL    NULL    NULL
+123    456     123     456     123
+123    4567890123456789        123     4567890123456789        123
 4567890123456789       4567890123456789        123     4567890123456789        123

 -- !query 93
-select count(*)
+select udf(count(*))
 from
-  (select t3.tenthous as x1, coalesce(t1.stringu1, t2.stringu1) as x2
+  (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2
    from tenk1 t1
-   left join tenk1 t2 on t1.unique1 = t2.unique1
-   join tenk1 t3 on t1.unique2 = t3.unique2) ss,
+   left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1)
+   join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss,
   tenk1 t4,
   tenk1 t5
-where t4.thousand = t5.unique1 and ss.x1 = t4.tenthous and ss.x2 = t5.stringu1
+where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1))
 -- !query 93 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 93 output
 1000

 -- !query 94
-select a.f1, b.f1, t.thousand, t.tenthous from
+select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from
   tenk1 t,
-  (select sum(f1)+1 as f1 from int4_tbl i4a) a,
-  (select sum(f1) as f1 from int4_tbl i4b) b
-where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous
+  (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a,
+  (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b
+where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous))
 -- !query 94 schema
-struct<f1:bigint,f1:bigint,thousand:int,tenthous:int>
+struct<CAST(udf(cast(f1 as string)) AS BIGINT):bigint,CAST(udf(cast(f1 as string)) AS BIGINT):bigint,CAST(udf(cast(thousand as string)) AS INT):int,CAST(udf(cast(tenthous as string)) AS INT):int>
 -- !query 94 output

 -2408,8 +2408,8  struct<f1:bigint,f1:bigint,thousand:int,tenthous:int>
 -- !query 95
 select * from
   j1_tbl full join
-  (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl
-  on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k
+  (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl
+  on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k)
 -- !query 95 schema
 struct<i:int,j:int,t:string,i:int,k:int>
 -- !query 95 output
 -2435,13 +2435,13  NULL    NULL    null    NULL    NULL

 -- !query 96
-select count(*) from
-  (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x
+select udf(count(*)) from
+  (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x
   left join
-  (select * from tenk1 y order by y.unique2) y
-  on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2
+  (select * from tenk1 y order by udf(y.unique2)) y
+  on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2
 -- !query 96 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 96 output
 10000

 -2507,7 +2507,7  struct<>

 -- !query 104
-select tt1.*, tt2.* from tt1 left join tt2 on tt1.joincol = tt2.joincol
+select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol)
 -- !query 104 schema
 struct<tt1_id:int,joincol:int,tt2_id:int,joincol:int>
 -- !query 104 output
 -2517,7 +2517,7  struct<tt1_id:int,joincol:int,tt2_id:int,joincol:int>

 -- !query 105
-select tt1.*, tt2.* from tt2 right join tt1 on tt1.joincol = tt2.joincol
+select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol))
 -- !query 105 schema
 struct<tt1_id:int,joincol:int,tt2_id:int,joincol:int>
 -- !query 105 output
 -2527,10 +2527,10  struct<tt1_id:int,joincol:int,tt2_id:int,joincol:int>

 -- !query 106
-select count(*) from tenk1 a, tenk1 b
-  where a.hundred = b.thousand and (b.fivethous % 10) < 10
+select udf(count(*)) from tenk1 a, tenk1 b
+  where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10)
 -- !query 106 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 106 output
 100000

 -2584,14 +2584,14  struct<>

 -- !query 113
-SELECT a.f1
+SELECT udf(udf(a.f1)) as f1
 FROM tt4 a
 LEFT JOIN (
         SELECT b.f1
-        FROM tt3 b LEFT JOIN tt3 c ON (b.f1 = c.f1)
-        WHERE c.f1 IS NULL
-) AS d ON (a.f1 = d.f1)
-WHERE d.f1 IS NULL
+        FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1)
+        WHERE udf(c.f1) IS NULL
+) AS d ON udf(a.f1) = d.f1
+WHERE udf(udf(d.f1)) IS NULL
 -- !query 113 schema
 struct<f1:int>
 -- !query 113 output
 -2621,7 +2621,7  struct<>

 -- !query 116
-select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2
+select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2))
 -- !query 116 schema
 struct<f1:int,f2:int,f1:int,f2:int>
 -- !query 116 output
 -2649,12 +2649,12  struct<>

 -- !query 119
-select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy,
-       xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx
+select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy,
+       udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx
 from yy
-     left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy
-     left join xx xxa on yya.pkxx = xxa.pkxx
-     left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx
+     left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy)
+     left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx))
+     left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx)
 -- !query 119 schema
 struct<yy_pkyy:int,yy_pkxx:int,yya_pkyy:int,xxa_pkxx:int,xxb_pkxx:int>
 -- !query 119 output
 -2693,9 +2693,9  struct<>

 -- !query 123
 select * from
-  zt2 left join zt3 on (f2 = f3)
-      left join zt1 on (f3 = f1)
-where f2 = 53
+  zt2 left join zt3 on (udf(f2) = udf(udf(f3)))
+      left join zt1 on (udf(udf(f3)) = udf(f1))
+where udf(f2) = 53
 -- !query 123 schema
 struct<f2:int,f3:int,f1:int>
 -- !query 123 output
 -2712,9 +2712,9  struct<>

 -- !query 125
 select * from
-  zt2 left join zt3 on (f2 = f3)
-      left join zv1 on (f3 = f1)
-where f2 = 53
+  zt2 left join zt3 on (f2 = udf(f3))
+      left join zv1 on (udf(f3) = f1)
+where udf(udf(f2)) = 53
 -- !query 125 schema
 struct<f2:int,f3:int,f1:int,junk:string>
 -- !query 125 output
 -2722,12 +2722,12  struct<f2:int,f3:int,f1:int,junk:string>

 -- !query 126
-select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred
-from tenk1 a left join tenk1 b on a.unique2 = b.tenthous
-where a.unique1 = 42 and
-      ((b.unique2 is null and a.ten = 2) or b.hundred = 3)
+select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred)
+from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous)
+where udf(a.unique1) = 42 and
+      ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3)))
 -- !query 126 schema
-struct<unique2:int,ten:int,tenthous:int,unique2:int,hundred:int>
+struct<CAST(udf(cast(unique2 as string)) AS INT):int,CAST(udf(cast(ten as string)) AS INT):int,CAST(udf(cast(tenthous as string)) AS INT):int,CAST(udf(cast(unique2 as string)) AS INT):int,CAST(udf(cast(hundred as string)) AS INT):int>
 -- !query 126 output

 -2749,7 +2749,7  struct<>

 -- !query 129
-select * from a left join b on i = x and i = y and x = i
+select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i)
 -- !query 129 schema
 struct<i:int,x:int,y:int>
 -- !query 129 output
 -2757,11 +2757,11  struct<i:int,x:int,y:int>

 -- !query 130
-select t1.q2, count(t2.*)
-from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1)
-group by t1.q2 order by 1
+select udf(t1.q2), udf(count(t2.*))
+from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1)
+group by udf(t1.q2) order by 1
 -- !query 130 schema
-struct<q2:bigint,count(q1, q2):bigint>
+struct<CAST(udf(cast(q2 as string)) AS BIGINT):bigint,CAST(udf(cast(count(q1, q2) as string)) AS BIGINT):bigint>
 -- !query 130 output
 -4567890123456789      0
 123    2
 -2770,11 +2770,11  struct<q2:bigint,count(q1, q2):bigint>

 -- !query 131
-select t1.q2, count(t2.*)
-from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1)
-group by t1.q2 order by 1
+select udf(udf(t1.q2)), udf(count(t2.*))
+from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1))
+group by udf(udf(t1.q2)) order by 1
 -- !query 131 schema
-struct<q2:bigint,count(q1, q2):bigint>
+struct<CAST(udf(cast(cast(udf(cast(q2 as string)) as bigint) as string)) AS BIGINT):bigint,CAST(udf(cast(count(q1, q2) as string)) AS BIGINT):bigint>
 -- !query 131 output
 -4567890123456789      0
 123    2
 -2783,13 +2783,13  struct<q2:bigint,count(q1, q2):bigint>

 -- !query 132
-select t1.q2, count(t2.*)
+select udf(t1.q2) as q2, udf(udf(count(t2.*)))
 from int8_tbl t1 left join
-  (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2
-  on (t1.q2 = t2.q1)
+  (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2
+  on (udf(t1.q2) = udf(t2.q1))
 group by t1.q2 order by 1
 -- !query 132 schema
-struct<q2:bigint,count(q1, q2):bigint>
+struct<q2:bigint,CAST(udf(cast(cast(udf(cast(count(q1, q2) as string)) as bigint) as string)) AS BIGINT):bigint>
 -- !query 132 output
 -4567890123456789      0
 123    2
 -2828,17 +2828,17  struct<>

 -- !query 136
-select c.name, ss.code, ss.b_cnt, ss.const
+select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const)
 from c left join
   (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const
    from a left join
-     (select count(1) as cnt, b.a from b group by b.a) as b_grp
-     on a.code = b_grp.a
+     (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp
+     on udf(a.code) = udf(udf(b_grp.a))
   ) as ss
-  on (c.a = ss.code)
+  on (udf(udf(c.a)) = udf(ss.code))
 order by c.name
 -- !query 136 schema
-struct<name:string,code:string,b_cnt:bigint,const:int>
+struct<CAST(udf(cast(name as string)) AS STRING):string,CAST(udf(cast(code as string)) AS STRING):string,CAST(udf(cast(b_cnt as string)) AS BIGINT):bigint,CAST(udf(cast(const as string)) AS INT):int>
 -- !query 136 output
 A      p       2       -1
 B      q       0       -1
 -2852,15 +2852,15  LEFT JOIN
 ( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM
     ( SELECT 1 as key3 ) sub3
     LEFT JOIN
-    ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM
+    ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM
         ( SELECT 1 as key5 ) sub5
         LEFT JOIN
         ( SELECT 2 as key6, 42 as value1 ) sub6
-        ON sub5.key5 = sub6.key6
+        ON sub5.key5 = udf(sub6.key6)
     ) sub4
-    ON sub4.key5 = sub3.key3
+    ON udf(sub4.key5) = sub3.key3
 ) sub2
-ON sub1.key1 = sub2.key3
+ON udf(udf(sub1.key1)) = udf(udf(sub2.key3))
 -- !query 137 schema
 struct<key1:int,key3:int,value2:int,value3:int>
 -- !query 137 output
 -2871,34 +2871,34  struct<key1:int,key3:int,value2:int,value3:int>
 SELECT * FROM
 ( SELECT 1 as key1 ) sub1
 LEFT JOIN
-( SELECT sub3.key3, value2, COALESCE(value2, 66) as value3 FROM
+( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM
     ( SELECT 1 as key3 ) sub3
     LEFT JOIN
     ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM
         ( SELECT 1 as key5 ) sub5
         LEFT JOIN
         ( SELECT 2 as key6, 42 as value1 ) sub6
-        ON sub5.key5 = sub6.key6
+        ON udf(udf(sub5.key5)) = sub6.key6
     ) sub4
     ON sub4.key5 = sub3.key3
 ) sub2
-ON sub1.key1 = sub2.key3
+ON sub1.key1 = udf(udf(sub2.key3))
 -- !query 138 schema
-struct<key1:int,key3:int,value2:int,value3:int>
+struct<key1:int,key3:int,CAST(udf(cast(value2 as string)) AS INT):int,value3:int>
 -- !query 138 output
 1      1       1       1

 -- !query 139
-SELECT qq, unique1
+SELECT udf(qq), udf(udf(unique1))
   FROM
-  ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1
+  ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1
   FULL OUTER JOIN
-  ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2
+  ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2
   USING (qq)
-  INNER JOIN tenk1 c ON qq = unique2
+  INNER JOIN tenk1 c ON udf(qq) = udf(unique2)
 -- !query 139 schema
-struct<qq:bigint,unique1:int>
+struct<CAST(udf(cast(qq as string)) AS BIGINT):bigint,CAST(udf(cast(cast(udf(cast(unique1 as string)) as int) as string)) AS INT):int>
 -- !query 139 output
 123    4596
 123    4596
 -2936,19 +2936,19  struct<>

 -- !query 143
-select nt3.id
+select udf(nt3.id)
 from nt3 as nt3
   left join
-    (select nt2.*, (nt2.b1 and ss1.a3) AS b3
+    (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3
      from nt2 as nt2
        left join
-         (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1
-         on ss1.id = nt2.nt1_id
+         (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1
+         on ss1.id = udf(udf(nt2.nt1_id))
     ) as ss2
-    on ss2.id = nt3.nt2_id
-where nt3.id = 1 and ss2.b3
+    on udf(ss2.id) = nt3.nt2_id
+where udf(nt3.id) = 1 and udf(ss2.b3)
 -- !query 143 schema
-struct<id:int>
+struct<CAST(udf(cast(id as string)) AS INT):int>
 -- !query 143 output
 1

 -3003,73 +3003,73  NULL    2147483647

 -- !query 146
-select count(*) from
-  tenk1 a join tenk1 b on a.unique1 = b.unique2
-  left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand
-  join int4_tbl on b.thousand = f1
+select udf(count(*)) from
+  tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2)
+  left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand))
+  join int4_tbl on udf(b.thousand) = f1
 -- !query 146 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 146 output
 10

 -- !query 147
-select b.unique1 from
-  tenk1 a join tenk1 b on a.unique1 = b.unique2
-  left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand
-  join int4_tbl i1 on b.thousand = f1
-  right join int4_tbl i2 on i2.f1 = b.tenthous
-  order by 1
+select udf(b.unique1) from
+  tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2)
+  left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand)
+  join int4_tbl i1 on udf(b.thousand) = udf(udf(f1))
+  right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous)
+  order by udf(1)
 -- !query 147 schema
-struct<unique1:int>
+struct<CAST(udf(cast(unique1 as string)) AS INT):int>
 -- !query 147 output
 NULL
 NULL
+0
 NULL
 NULL
-0

 -- !query 148
 select * from
 (
-  select unique1, q1, coalesce(unique1, -1) + q1 as fault
-  from int8_tbl left join tenk1 on (q2 = unique2)
+  select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault
+  from int8_tbl left join tenk1 on (udf(q2) = udf(unique2))
 ) ss
-where fault = 122
-order by fault
+where udf(fault) = udf(122)
+order by udf(fault)
 -- !query 148 schema
-struct<unique1:int,q1:bigint,fault:bigint>
+struct<CAST(udf(cast(unique1 as string)) AS INT):int,CAST(udf(cast(q1 as string)) AS BIGINT):bigint,fault:bigint>
 -- !query 148 output
 NULL   123     122

 -- !query 149
-select q1, unique2, thousand, hundred
-  from int8_tbl a left join tenk1 b on q1 = unique2
-  where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123)
+select udf(q1), udf(unique2), udf(thousand), udf(hundred)
+  from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2)
+  where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123)))
 -- !query 149 schema
-struct<q1:bigint,unique2:int,thousand:int,hundred:int>
+struct<CAST(udf(cast(q1 as string)) AS BIGINT):bigint,CAST(udf(cast(unique2 as string)) AS INT):int,CAST(udf(cast(thousand as string)) AS INT):int,CAST(udf(cast(hundred as string)) AS INT):int>
 -- !query 149 output

 -- !query 150
-select f1, unique2, case when unique2 is null then f1 else 0 end
-  from int4_tbl a left join tenk1 b on f1 = unique2
-  where (case when unique2 is null then f1 else 0 end) = 0
+select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end
+  from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2))
+  where (case when udf(unique2) is null then udf(f1) else 0 end) = 0
 -- !query 150 schema
-struct<f1:int,unique2:int,CASE WHEN (unique2 IS NULL) THEN f1 ELSE 0 END:int>
+struct<CAST(udf(cast(f1 as string)) AS INT):int,CAST(udf(cast(unique2 as string)) AS INT):int,CASE WHEN (CAST(udf(cast(cast(udf(cast(unique2 as string)) as int) as string)) AS INT) IS NULL) THEN CAST(udf(cast(f1 as string)) AS INT) ELSE 0 END:int>
 -- !query 150 output
 0      0       0

 -- !query 151
-select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand)
-  from tenk1 a left join tenk1 b on b.thousand = a.unique1                        left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand)
-  where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44
+select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand))
+  from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1                       left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand))
+  where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44)
 -- !query 151 schema
-struct<unique1:int,unique1:int,unique1:int,coalesce(twothousand, twothousand):int>
+struct<CAST(udf(cast(unique1 as string)) AS INT):int,CAST(udf(cast(unique1 as string)) AS INT):int,CAST(udf(cast(unique1 as string)) AS INT):int,CAST(udf(cast(coalesce(twothousand, twothousand) as string)) AS INT):int>
 -- !query 151 output

 -3078,11 +3078,11  struct<unique1:int,unique1:int,unique1:int,coalesce(twothousand, twothousand):in
 select * from
   text_tbl t1
   inner join int8_tbl i8
-  on i8.q2 = 456
+  on udf(i8.q2) = udf(udf(456))
   right join text_tbl t2
-  on t1.f1 = 'doh!'
+  on udf(t1.f1) = udf(udf('doh!'))
   left join int4_tbl i4
-  on i8.q1 = i4.f1
+  on udf(udf(i8.q1)) = i4.f1
 -- !query 152 schema
 struct<f1:string,q1:bigint,q2:bigint,f1:string,f1:int>
 -- !query 152 output
 -3092,10 +3092,10  doh!    123     456     hi de ho neighbor       NULL

 -- !query 153
 select * from
-  (select 1 as id) as xx
+  (select udf(udf(1)) as id) as xx
   left join
-    (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id))
-  on (xx.id = coalesce(yy.id))
+    (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id)))
+  on (xx.id = udf(udf(coalesce(yy.id))))
 -- !query 153 schema
 struct<id:int,unique1:int,unique2:int,two:int,four:int,ten:int,twenty:int,hundred:int,thousand:int,twothousand:int,fivethous:int,tenthous:int,odd:int,even:int,stringu1:string,stringu2:string,string4:string,id:int>
 -- !query 153 output
 -3103,11 +3103,11  struct<id:int,unique1:int,unique2:int,two:int,four:int,ten:int,twenty:int,hundre

 -- !query 154
-select a.q2, b.q1
-  from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1)
-  where coalesce(b.q1, 1) > 0
+select udf(a.q2), udf(b.q1)
+  from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1)
+  where udf(udf(coalesce(b.q1, 1)) > 0)
 -- !query 154 schema
-struct<q2:bigint,q1:bigint>
+struct<CAST(udf(cast(q2 as string)) AS BIGINT):bigint,CAST(udf(cast(q1 as string)) AS BIGINT):bigint>
 -- !query 154 output
 -4567890123456789      NULL
 123    123
 -3142,7 +3142,7  struct<>

 -- !query 157
-select p.* from parent p left join child c on (p.k = c.k)
+select p.* from parent p left join child c on (udf(p.k) = udf(c.k))
 -- !query 157 schema
 struct<k:int,pd:int>
 -- !query 157 output
 -3153,8 +3153,8  struct<k:int,pd:int>

 -- !query 158
 select p.*, linked from parent p
-  left join (select c.*, true as linked from child c) as ss
-  on (p.k = ss.k)
+  left join (select c.*, udf(udf(true)) as linked from child c) as ss
+  on (udf(p.k) = udf(udf(ss.k)))
 -- !query 158 schema
 struct<k:int,pd:int,linked:boolean>
 -- !query 158 output
 -3165,8 +3165,8  struct<k:int,pd:int,linked:boolean>

 -- !query 159
 select p.* from
-  parent p left join child c on (p.k = c.k)
-  where p.k = 1 and p.k = 2
+  parent p left join child c on (udf(p.k) = c.k)
+  where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2))
 -- !query 159 schema
 struct<k:int,pd:int>
 -- !query 159 output
 -3175,8 +3175,8  struct<k:int,pd:int>

 -- !query 160
 select p.* from
-  (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k
-  where p.k = 1 and p.k = 2
+  (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k)
+  where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2))
 -- !query 160 schema
 struct<k:int,pd:int>
 -- !query 160 output
 -3204,7 +3204,7  struct<>

 -- !query 163
-SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0)
+SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0)
 -- !query 163 schema
 struct<id:int,a_id:int,id:int>
 -- !query 163 output
 -3212,7 +3212,7  struct<id:int,a_id:int,id:int>

 -- !query 164
-SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0)
+SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0)
 -- !query 164 schema
 struct<id:int,a_id:int>
 -- !query 164 output
 -3231,13 +3231,13  struct<>

 -- !query 166
 SELECT * FROM
-    (SELECT 1 AS x) ss1
+    (SELECT udf(1) AS x) ss1
   LEFT JOIN
-    (SELECT q1, q2, COALESCE(dat1, q1) AS y
-     FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2
+    (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y
+     FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2
   ON true
 -- !query 166 schema
-struct<x:int,q1:bigint,q2:bigint,y:bigint>
+struct<x:int,CAST(udf(cast(q1 as string)) AS BIGINT):bigint,CAST(udf(cast(q2 as string)) AS BIGINT):bigint,y:bigint>
 -- !query 166 output
 1      123     456     123
 1      123     4567890123456789        123
 -3248,27 +3248,27  struct<x:int,q1:bigint,q2:bigint,y:bigint>

 -- !query 167
 select * from
-  int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1
+  int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1)
 -- !query 167 schema
 struct<>
 -- !query 167 output
 org.apache.spark.sql.AnalysisException
-Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 63
+Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 72

 -- !query 168
 select * from
-  int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1
+  int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1)
 -- !query 168 schema
 struct<>
 -- !query 168 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63
+cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72

 -- !query 169
 select * from
-  int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1
+  int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1))
 -- !query 169 schema
 struct<q1:bigint,q2:bigint,f1:int,ff:int>
 -- !query 169 output
 -3276,69 +3276,69  struct<q1:bigint,q2:bigint,f1:int,ff:int>

 -- !query 170
-select t1.uunique1 from
-  tenk1 t1 join tenk2 t2 on t1.two = t2.two
+select udf(t1.uunique1) from
+  tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two)
 -- !query 170 schema
 struct<>
 -- !query 170 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7
+cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11

 -- !query 171
-select t2.uunique1 from
-  tenk1 t1 join tenk2 t2 on t1.two = t2.two
+select udf(udf(t2.uunique1)) from
+  tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two
 -- !query 171 schema
 struct<>
 -- !query 171 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7
+cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 15

 -- !query 172
-select uunique1 from
-  tenk1 t1 join tenk2 t2 on t1.two = t2.two
+select udf(uunique1) from
+  tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two)
 -- !query 172 schema
 struct<>
 -- !query 172 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7
+cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11

 -- !query 173
-select f1,g from int4_tbl a, (select f1 as g) ss
+select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss
 -- !query 173 schema
 struct<>
 -- !query 173 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`f1`' given input columns: []; line 1 pos 37
+cannot resolve '`f1`' given input columns: []; line 1 pos 55

 -- !query 174
-select f1,g from int4_tbl a, (select a.f1 as g) ss
+select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss
 -- !query 174 schema
 struct<>
 -- !query 174 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`a.f1`' given input columns: []; line 1 pos 37
+cannot resolve '`a.f1`' given input columns: []; line 1 pos 42

 -- !query 175
-select f1,g from int4_tbl a cross join (select f1 as g) ss
+select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss
 -- !query 175 schema
 struct<>
 -- !query 175 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`f1`' given input columns: []; line 1 pos 47
+cannot resolve '`f1`' given input columns: []; line 1 pos 61

 -- !query 176
-select f1,g from int4_tbl a cross join (select a.f1 as g) ss
+select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss
 -- !query 176 schema
 struct<>
 -- !query 176 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`a.f1`' given input columns: []; line 1 pos 47
+cannot resolve '`a.f1`' given input columns: []; line 1 pos 60

 -- !query 177
 -3383,8 +3383,8  struct<>

 -- !query 182
 select * from j1
-inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2
-where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1
+inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2)
+where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1
 -- !query 182 schema
 struct<id1:int,id2:int,id1:int,id2:int>
 -- !query 182 output
```

</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25371 from huaxingao/spark-28393.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-19 20:10:56 +09:00
Wenchen Fan 97dc4c0bfc [SPARK-28744][SQL][TEST] rename SharedSQLContext to SharedSparkSession
## What changes were proposed in this pull request?

The Spark SQL test framework needs to support 2 kinds of tests:
1. tests inside Spark to test Spark itself (extends `SparkFunSuite`)
2. test outside of Spark to test Spark applications (introduced at b57ed2245c)

The class hierarchy of the major testing traits:
![image](https://user-images.githubusercontent.com/3182036/63088526-c0f0af80-bf87-11e9-9bed-c144c2486da9.png)

`PlanTestBase`, `SQLTestUtilsBase` and `SharedSparkSession` intentionally don't extend `SparkFunSuite`, so that they can be used for tests outside of Spark. Tests in Spark should extends `QueryTest` and/or `SharedSQLContext` in most cases.

However, the name is a little confusing. As a result, some test suites extend `SharedSparkSession` instead of `SharedSQLContext`. `SharedSparkSession` doesn't work well with `SparkFunSuite` as it doesn't have the special handling of thread auditing in `SharedSQLContext`. For example, you will see a warning starting with `===== POSSIBLE THREAD LEAK IN SUITE` when you run `DataFrameSelfJoinSuite`.

This PR proposes to rename `SharedSparkSession` to `SharedSparkSessionBase`, and rename `SharedSQLContext` to `SharedSparkSession`.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25463 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-19 19:01:56 +08:00
Peter Toth f999e00e9f [SPARK-28356][SHUFFLE][FOLLOWUP] Fix case with different pre-shuffle partition numbers
### What changes were proposed in this pull request?

This PR reverts some of the latest changes in `ReduceNumShufflePartitions` to fix the case when there are different pre-shuffle partition numbers in the plan. Please see the new UT for an example.

### Why are the changes needed?
Eliminate a bug.

### Does this PR introduce any user-facing change?
Yes, some queries that failed will succeed now.

### How was this patch tested?
Added new UT.

Closes #25479 from peter-toth/SPARK-28356-followup.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-19 15:53:43 +08:00
Eyal Zituny d75a11d059 [SPARK-27330][SS] support task abort in foreach writer
## What changes were proposed in this pull request?
in order to address cases where foreach writer task is failing without calling the close() method, (for example when a task is interrupted) added the option to implement an abort() method that will be called when the task is aborted. users should handle resource cleanup (such as connections) in the abort() method

## How was this patch tested?
update existing unit tests.

Closes #24382 from eyalzit/SPARK-27330-foreach-writer-abort.

Lead-authored-by: Eyal Zituny <eyal.zituny@equalum.io>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Co-authored-by: eyalzit <eyal.zituny@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-19 14:12:48 +08:00
shivusondur c96b6154b7 [SPARK-28390][SQL][PYTHON][TESTS][FOLLOW-UP] Update the TODO with actual blocking JIRA IDs
## What changes were proposed in this pull request?
 only todo message updated. Need to add udf() for GroupBy Tests, after resolving following jira
[SPARK-28386] and [SPARK-26741]

## How was this patch tested?
NA, only TODO message updated.

Closes #25415 from shivusondur/jiraFollowup.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-19 13:01:39 +09:00
WeichenXu 4ddad79060 [SPARK-28598][SQL] Few date time manipulation functions does not provide versions supporting Column as input through the Dataframe API
## What changes were proposed in this pull request?

Add following functions:
```
def add_months(startDate: Column, numMonths: Column): Column
def date_add(start: Column, days: Column): Column
def date_sub(start: Column, days: Column): Column
```

## How was this patch tested?

UT.

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25334 from WeichenXu123/datefunc_impr.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-19 11:41:13 +09:00
Dongjoon Hyun f0834d3a7f Revert "[SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server"
This reverts commit efbb035902.
2019-08-18 16:54:24 -07:00
Yuming Wang c308ab5a29 [MINOR][SQL] Make analysis error msg more meaningful on DISTINCT queries
## What changes were proposed in this pull request?

This PR makes analysis error messages more meaningful when the function does not support the modifier DISTINCT:
```sql
postgres=# select upper(distinct a) from (values('a'), ('b')) v(a);
ERROR:  DISTINCT specified, but upper is not an aggregate function
LINE 1: select upper(distinct a) from (values('a'), ('b')) v(a);

spark-sql> select upper(distinct a) from (values('a'), ('b')) v(a);
Error in query: upper does not support the modifier DISTINCT; line 1 pos 7
spark-sql>
```

After this pr:
```sql
spark-sql> select upper(distinct a) from (values('a'), ('b')) v(a);
Error in query: DISTINCT specified, but upper is not an aggregate function; line 1 pos 7
spark-sql>

```

## How was this patch tested?

Unit test

Closes #25486 from wangyum/DISTINCT.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-18 08:36:01 -07:00
Yuming Wang efbb035902 [SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server
## What changes were proposed in this pull request?

This PR build a test framework that directly re-run all the tests in `SQLQueryTestSuite` via Thrift Server. But it's a little different from `SQLQueryTestSuite`:
1. Can not support [UDF testing](44e607e921/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L293-L297)).
2. Can not support `DESC` command and `SHOW` command because `SQLQueryTestSuite` [formatted the output](1882912cca/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala (L38-L50).).

When building this framework, found two bug:
[SPARK-28624](https://issues.apache.org/jira/browse/SPARK-28624): `make_date` is inconsistent when reading from table
[SPARK-28611](https://issues.apache.org/jira/browse/SPARK-28611): Histogram's height is different

found two features that ThriftServer can not support:
[SPARK-28636](https://issues.apache.org/jira/browse/SPARK-28636): ThriftServer can not support decimal type with negative scale
[SPARK-28637](https://issues.apache.org/jira/browse/SPARK-28637): ThriftServer can not support interval type

Also, found two inconsistent behavior:
[SPARK-28620](https://issues.apache.org/jira/browse/SPARK-28620): Double type returned for float type in Beeline/JDBC
[SPARK-28619](https://issues.apache.org/jira/browse/SPARK-28619):  The golden result file is different when tested by `bin/spark-sql`

## How was this patch tested?

N/A

Closes #25373 from wangyum/SPARK-28527.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-17 19:12:50 -07:00
Gengliang Wang 92bfd9a317 [SPARK-28757][SQL] File table location should include both values of option path and paths
### What changes were proposed in this pull request?
If both options `path` and `paths` are passed to file data source v2, both values of the options should be included as the target paths.

### Why are the changes needed?
In V1 implementation, file table location includes both values of option `path` and `paths`.
In the refactoring of https://github.com/apache/spark/pull/24025, the value of option `path` is ignored if "paths" are specified. We should make it consistent with V1.

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

### How was this patch tested?
Unit test

Closes #25473 from gengliangwang/fixPathOption.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-16 22:27:27 +08:00
pavithra c48e381214 [SPARK-28671][SQL] Throw NoSuchPermanentFunctionException for a non-exsistent permanent function in dropFunction/alterFunction
## What changes were proposed in this pull request?
**Before Fix**
When a non existent permanent function is dropped, generic NoSuchFunctionException was thrown.- which printed "This function is neither a registered temporary function nor a permanent function registered in the database" .
This creates a ambiguity when a temp function in the same name exist.

**After Fix**
 NoSuchPermanentFunctionException will be thrown, which will print
"NoSuchPermanentFunctionException:Function not found in database "

## How was this patch tested?
Unit test was run and corrected the UT.

Closes #25394 from PavithraRamachandran/funcIssue.

Lead-authored-by: pavithra <pavi.rams@gmail.com>
Co-authored-by: pavithraramachandran <pavi.rams@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-16 22:46:04 +09:00
Maxim Gekk 96ca734fb7 [SPARK-28745][SQL][TEST] Add benchmarks for extract()
## What changes were proposed in this pull request?

Added new benchmark `ExtractBenchmark` for the `EXTRACT(field FROM source)` function. It was executed on all currently supported values of the `field` argument:  `MILLENNIUM`, `CENTURY`, `DECADE`, `YEAR`, `ISOYEAR`, `QUARTER`, `MONTH`, `WEEK`, `DAY`, `DAYOFWEEK`, `HOUR`, `MINUTE`, `SECOND`, `MILLISECONDS`, `MICROSECONDS`, `EPOCH`. The `cast(id as timestamp)` was taken as the `source` argument.

## How was this patch tested?

By running the benchmark via:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.ExtractBenchmark"
```

Closes #25462 from MaxGekk/extract-benchmark.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-15 12:44:36 -07:00
Yuming Wang 1b416a0c77 [SPARK-27592][SQL] Set the bucketed data source table SerDe correctly
## What changes were proposed in this pull request?

Hive using incorrect **InputFormat**(`org.apache.hadoop.mapred.SequenceFileInputFormat`) to read Spark's **Parquet** bucketed data source table.
Spark side:
```sql
spark-sql> CREATE TABLE t (c1 INT, c2 INT) USING parquet CLUSTERED BY (c1) SORTED BY (c1) INTO 2 BUCKETS;
2019-04-29 17:52:05 WARN  HiveExternalCatalog:66 - Persisting bucketed data source table `default`.`t` into Hive metastore in Spark SQL specific format, which is NOT compatible with Hive.
spark-sql> DESC FORMATTED t;
c1	int	NULL
c2	int	NULL

# Detailed Table Information
Database	default
Table	t
Owner	yumwang
Created Time	Mon Apr 29 17:52:05 CST 2019
Last Access	Thu Jan 01 08:00:00 CST 1970
Created By	Spark 2.4.0
Type	MANAGED
Provider	parquet
Num Buckets	2
Bucket Columns	[`c1`]
Sort Columns	[`c1`]
Table Properties	[transient_lastDdlTime=1556531525]
Location	file:/user/hive/warehouse/t
Serde Library	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat	org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties	[serialization.format=1]
```
Hive side:
```sql
hive> DESC FORMATTED t;
OK
# col_name            	data_type           	comment

c1                  	int
c2                  	int

# Detailed Table Information
Database:           	default
Owner:              	root
CreateTime:         	Wed May 08 03:38:46 GMT-07:00 2019
LastAccessTime:     	UNKNOWN
Retention:          	0
Location:           	file:/user/hive/warehouse/t
Table Type:         	MANAGED_TABLE
Table Parameters:
	bucketing_version   	spark
	spark.sql.create.version	3.0.0-SNAPSHOT
	spark.sql.sources.provider	parquet
	spark.sql.sources.schema.bucketCol.0	c1
	spark.sql.sources.schema.numBucketCols	1
	spark.sql.sources.schema.numBuckets	2
	spark.sql.sources.schema.numParts	1
	spark.sql.sources.schema.numSortCols	1
	spark.sql.sources.schema.part.0	{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}
	spark.sql.sources.schema.sortCol.0	c1
	transient_lastDdlTime	1557311926

# Storage Information
SerDe Library:      	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat:        	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat:       	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
Compressed:         	No
Num Buckets:        	-1
Bucket Columns:     	[]
Sort Columns:       	[]
Storage Desc Params:
	path                	file:/user/hive/warehouse/t
	serialization.format	1
```

So it's non-bucketed table at Hive side. This pr set the `SerDe` correctly so Hive can read these tables.

Related code:
33f3c48cac/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L976-L990)
f9776e3892/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala (L444-L459)

## How was this patch tested?

unit tests

Closes #24486 from wangyum/SPARK-27592.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-15 17:21:13 +08:00
Burak Yavuz 0526529b31 [SPARK-28666] Support saveAsTable for V2 tables through Session Catalog
## What changes were proposed in this pull request?

We add support for the V2SessionCatalog for saveAsTable, such that V2 tables can plug in and leverage existing DataFrameWriter.saveAsTable APIs to write and create tables through the session catalog.

## How was this patch tested?

Unit tests. A lot of tests broke under hive when things were not working properly under `ResolveTables`, therefore I believe the current set of tests should be sufficient in testing the table resolution and read code paths.

Closes #25402 from brkyvz/saveAsV2.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-15 12:29:34 +08:00
Maxim Gekk 3a4afce96c [SPARK-28687][SQL] Support epoch, isoyear, milliseconds and microseconds at extract()
## What changes were proposed in this pull request?

In the PR, I propose new expressions `Epoch`, `IsoYear`, `Milliseconds` and `Microseconds`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT):

1. `epoch` - the number of seconds since 1970-01-01 00:00:00 local time in microsecond precision.
2. `isoyear` - the ISO 8601 week-numbering year that the date falls in. Each ISO 8601 week-numbering year begins with the Monday of the week containing the 4th of January.
3. `milliseconds` - the seconds field including fractional parts multiplied by 1,000.
4. `microseconds` - the seconds field including fractional parts multiplied by 1,000,000.

Here are examples:
```sql
spark-sql> SELECT EXTRACT(EPOCH FROM TIMESTAMP '2019-08-11 19:07:30.123456');
1565550450.123456
spark-sql> SELECT EXTRACT(ISOYEAR FROM DATE '2006-01-01');
2005
spark-sql> SELECT EXTRACT(MILLISECONDS FROM TIMESTAMP '2019-08-11 19:07:30.123456');
30123.456
spark-sql> SELECT EXTRACT(MICROSECONDS FROM TIMESTAMP '2019-08-11 19:07:30.123456');
30123456
```

## How was this patch tested?

Added new tests to `DateExpressionsSuite`, and uncommented existing tests in `extract.sql` and `pgSQL/date.sql`.

Closes #25408 from MaxGekk/extract-ext3.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-14 08:44:44 -07:00
xy_xin 2eeb25e52d [SPARK-28351][SQL] Support DELETE in DataSource V2
## What changes were proposed in this pull request?

This pr adds DELETE support for V2 datasources. As a first step, this pr only support delete by source filters:
```scala
void delete(Filter[] filters);
```
which could not deal with complicated cases like subqueries.

Since it's uncomfortable to embed the implementation of DELETE in the current V2 APIs, a new mix-in of datasource is added, which is called `SupportsMaintenance`, similar to `SupportsRead` and `SupportsWrite`.  A datasource which can be maintained means we can perform DELETE/UPDATE/MERGE/OPTIMIZE on the datasource, as long as the datasource implements the necessary mix-ins.

## How was this patch tested?

new test case.

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25115 from xianyinxin/SPARK-28351.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-14 23:38:45 +08:00
John Zhuge 391c7e8f2e [SPARK-27739][SQL] df.persist should save stats from optimized plan
## What changes were proposed in this pull request?

CacheManager.cacheQuery saves the stats from the optimized plan to cache.

## How was this patch tested?

Existing testss.

Closes #24623 from jzhuge/SPARK-27739.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-14 19:49:53 +08:00
Edgar Rodriguez 598fcbe5ed [SPARK-28265][SQL] Add renameTable to TableCatalog API
## What changes were proposed in this pull request?

This PR adds the `renameTable` call to the `TableCatalog` API, as described in the [Table Metadata API SPIP](https://docs.google.com/document/d/1zLFiA1VuaWeVxeTDXNg8bL6GP3BVoOZBkewFtEnjEoo/edit#heading=h.m45webtwxf2d).

This PR is related to: https://github.com/apache/spark/pull/24246

## How was this patch tested?

Added  unit tests and contract tests.

Closes #25206 from edgarRd/SPARK-28265-add-rename-table-catalog-api.

Authored-by: Edgar Rodriguez <edgar.rd@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-14 14:24:13 +08:00
Dilip Biswal 331f2657d9 [SPARK-27768][SQL] Support Infinity/NaN-related float/double literals case-insensitively
## What changes were proposed in this pull request?
Here is the problem description from the JIRA.
```
When the inputs contain the constant 'infinity', Spark SQL does not generate the expected results.

SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('1'), (CAST('infinity' AS DOUBLE))) v(x);
SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('infinity'), ('1')) v(x);
SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('infinity'), ('infinity')) v(x);
SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('-infinity'), ('infinity')) v(x);
 The root cause: Spark SQL does not recognize the special constants in a case insensitive way. In PostgreSQL, they are recognized in a case insensitive way.

Link: https://www.postgresql.org/docs/9.3/datatype-numeric.html
```

In this PR, the casting code is enhanced to handle these `special` string literals in case insensitive manner.

## How was this patch tested?
Added tests in CastSuite and modified existing test suites.

Closes #25331 from dilipbiswal/double_infinity.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 16:48:30 -07:00
Maxim Gekk 3d85c54895 [SPARK-28700][SQL] Use DECIMAL type for sec in make_timestamp()
## What changes were proposed in this pull request?

Changed type of `sec` argument in the `make_timestamp()` function from `DOUBLE` to `DECIMAL(8, 6)`. The scale is set to 6 to cover microsecond fractions, and the precision is 2 digits for seconds + 6 digits for microsecond fraction. New type prevents losing precision in some cases, for example:

Before:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 58.000001);
2019-08-12 00:00:58
```

After:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 58.000001);
2019-08-12 00:00:58.000001
```

Also switching to `DECIMAL` fixes rounding `sec` towards "nearest neighbor" unless both neighbors are equidistant, in which case round up. For example:

Before:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 0.1234567);
2019-08-12 00:00:00.123456
```

After:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 0.1234567);
2019-08-12 00:00:00.123457
```

## How was this patch tested?

This was tested by `DateExpressionsSuite` and `pgSQL/timestamp.sql`.

Closes #25421 from MaxGekk/make_timestamp-decimal.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 15:51:50 -07:00
Maxim Gekk f04a766946 [SPARK-28718][SQL] Support field synonyms at extract
## What changes were proposed in this pull request?

In the PR, I propose additional synonyms for the `field` argument of `extract` supported by PostgreSQL. The `extract.sql` is updated to check all supported values of the `field` argument. The list of synonyms was taken from https://github.com/postgres/postgres/blob/master/src/backend/utils/adt/datetime.c .

## How was this patch tested?

By running `extract.sql` via:
```
$ build/sbt "sql/test-only *SQLQueryTestSuite -- -z extract.sql"
```

Closes #25438 from MaxGekk/extract-field-synonyms.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 15:36:28 -07:00
Yuming Wang 13b62f31cd [SPARK-28708][SQL] IsolatedClientLoader will not load hive classes from application jars on JDK9+
## What changes were proposed in this pull request?

We have 8 test cases in `HiveSparkSubmitSuite` still fail with `java.lang.ClassNotFoundException` when running on JDK9+:
```
[info] - SPARK-18989: DESC TABLE should not fail with format class not found *** FAILED *** (9 seconds, 927 milliseconds)
[info]   spark-submit returned with exit code 1.
[info]   Command line: './bin/spark-submit' '--class' 'org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE' '--name' 'SPARK-18947' '--master' 'local-cluster[2,1,1024]' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' '--jars' '/root/.m2/repository/org/apache/hive/hive-contrib/2.3.6-SNAPSHOT/hive-contrib-2.3.6-SNAPSHOT.jar' 'file:/root/opensource/spark/target/tmp/spark-36d27542-7b82-4962-a362-bb51ef3e457d/testJar-1565682620744.jar'
[info]
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: An illegal reflective access operation has occurred
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/root/opensource/spark/common/unsafe/target/scala-2.12/classes/) to constructor java.nio.DirectByteBuffer(long,int)
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: All illegal access operations will be denied in a future release
[info]   2019-08-13 00:50:28.31 - stderr> Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/metadata/HiveException
[info]   2019-08-13 00:50:28.31 - stderr> 	at java.base/java.lang.Class.getDeclaredConstructors0(Native Method)
[info]   2019-08-13 00:50:28.31 - stderr> 	at java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:3138)
[info]   2019-08-13 00:50:28.31 - stderr> 	at java.base/java.lang.Class.getConstructors(Class.java:1944)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:294)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:410)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:305)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:68)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:67)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$databaseExists$1(HiveExternalCatalog.scala:221)
[info]   2019-08-13 00:50:28.31 - stderr> 	at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:221)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:139)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:129)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:42)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.HiveSessionStateBuilder.$anonfun$catalog$1(HiveSessionStateBuilder.scala:57)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog$lzycompute(SessionCatalog.scala:91)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog(SessionCatalog.scala:91)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.databaseExists(SessionCatalog.scala:244)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.requireDbExists(SessionCatalog.scala:178)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:317)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.CreateTableCommand.run(tables.scala:132)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:213)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3431)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3427)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:213)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:95)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:653)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE$.main(HiveSparkSubmitSuite.scala:829)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE.main(HiveSparkSubmitSuite.scala)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:920)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:179)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:202)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:89)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:999)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1008)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
[info]   2019-08-13 00:50:28.311 - stderr> Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.metadata.HiveException
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.net.URLClassLoader.findClass(URLClassLoader.java:471)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:588)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:250)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:239)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:521)
[info]   2019-08-13 00:50:28.311 - stderr> 	... 48 more
```

Note that this pr fixes `java.lang.ClassNotFoundException`, but the test will fail again with a different reason, the Hive-side `java.lang.ClassCastException` which will be resolved in the official Hive 2.3.6 release.
```
[info] - SPARK-18989: DESC TABLE should not fail with format class not found *** FAILED *** (7 seconds, 649 milliseconds)
[info]   spark-submit returned with exit code 1.
[info]   Command line: './bin/spark-submit' '--class' 'org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE' '--name' 'SPARK-18947' '--master' 'local-cluster[2,1,1024]' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' '--jars' '/Users/dongjoon/.ivy2/cache/org.apache.hive/hive-contrib/jars/hive-contrib-2.3.5.jar' 'file:/Users/dongjoon/PRS/PR-25429/target/tmp/spark-48b7c936-0ec2-4311-9fb5-0de4bf86a0eb/testJar-1565710418275.jar'
[info]
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: An illegal reflective access operation has occurred
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dongjoon/PRS/PR-25429/common/unsafe/target/scala-2.12/classes/) to constructor java.nio.DirectByteBuffer(long,int)
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: All illegal access operations will be denied in a future release
[info]   2019-08-13 08:33:43.59 - stderr> Exception in thread "main" org.apache.spark.sql.AnalysisException: java.lang.ClassCastException: class jdk.internal.loader.ClassLoaders$AppClassLoader cannot be cast to class java.net.URLClassLoader (jdk.internal.loader.ClassLoaders$AppClassLoader and java.net.URLClassLoader are in module java.base of loader 'bootstrap');
[info]   2019-08-13 08:33:43.59 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109)
```

## How was this patch tested?

manual tests:

1. Install [Hive 2.3.6-SNAPSHOT](https://github.com/wangyum/hive/tree/HIVE-21584-branch-2.3) to local maven repository:
```
mvn clean install -DskipTests=true
```
2. Upgrade our built-in Hive to 2.3.6-SNAPSHOT, you can checkout [this branch](https://github.com/wangyum/spark/tree/SPARK-28708-Hive-2.3.6) to test.
3. Test with hadoop-3.2:
```
build/sbt "hive/test-only *. HiveSparkSubmitSuite" -Phive -Phadoop-3.2 -Phive-thriftserver
...
[info] Run completed in 3 minutes, 8 seconds.
[info] Total number of tests run: 11
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 11, failed 0, canceled 3, ignored 0, pending 0
[info] All tests passed.
```

Closes #25429 from wangyum/SPARK-28708.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 11:21:19 -07:00
Yuming Wang c81da276ba [SPARK-28714][SQL][TEST] Add hive.aux.jars.path test for spark-sql shell
## What changes were proposed in this pull request?

`Utilities.addToClassPath` has been changed since [HIVE-22096](https://issues.apache.org/jira/browse/HIVE-22096), but we use it to add plugin jars:
128ea37bda/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala (L144-L147)

This PR add test for `spark-sql` adding plugin jars.

## How was this patch tested?

N/A

Closes #25435 from wangyum/SPARK-28714.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 09:19:58 -07:00
Liang-Chi Hsieh e6a0385289 [SPARK-28422][SQL][PYTHON] GROUPED_AGG pandas_udf should work without group by clause
## What changes were proposed in this pull request?

A GROUPED_AGG pandas python udf can't work, if without group by clause, like `select udf(id) from table`.

This doesn't match with aggregate function like sum, count..., and also dataset API like `df.agg(udf(df['id']))`.

When we parse a udf (or an aggregate function) like that from SQL syntax, it is known as a function in a project. `GlobalAggregates` rule in analysis makes such project as aggregate, by looking for aggregate expressions. At the moment, we should also look for GROUPED_AGG pandas python udf.

## How was this patch tested?

Added tests.

Closes #25352 from viirya/SPARK-28422.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-14 00:32:33 +09:00
Xingbo Jiang 3249c7ab49 [SPARK-28706][SQL] Allow cast null type to any types
## What changes were proposed in this pull request?

#25242 proposed to disallow upcasting complex data types to string type, however, upcasting from null type to any types should still be safe.

## How was this patch tested?

Add corresponding case in `CastSuite`.

Closes #25425 from jiangxb1987/nullToString.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-13 19:02:04 +08:00
Yuming Wang 9a7f29023e [SPARK-28383][SQL] SHOW CREATE TABLE is not supported on a temporary view
## What changes were proposed in this pull request?
It throws `Table or view not found` when showing  temporary views:
```sql
spark-sql> CREATE TEMPORARY VIEW temp_view AS SELECT 1 AS a;
spark-sql> show create table temp_view;
Error in query: Table or view 'temp_view' not found in database 'default';
```
It's not easy to support temporary views. This pr changed it to throws `SHOW CREATE TABLE is not supported on a temporary view`:
```sql
spark-sql> CREATE TEMPORARY VIEW temp_view AS SELECT 1 AS a;
spark-sql> show create table temp_view;
Error in query: SHOW CREATE TABLE is not supported on a temporary view: temp_view;
```

## How was this patch tested?

unit tests

Closes #25149 from wangyum/SPARK-28383.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 21:01:19 -07:00
Yuming Wang 016e1b491c [SPARK-28703][SQL][TEST] Skip HiveExternalCatalogVersionsSuite and 3 tests in HiveSparkSubmitSuite at JDK9+
## What changes were proposed in this pull request?
This PR skip more test when testing with `JAVA_9` or later:
1. Skip `HiveExternalCatalogVersionsSuite` when testing with `JAVA_9` or later because our previous version does not support `JAVA_9` or later.

2. Skip 3 tests in `HiveSparkSubmitSuite` because the `spark.sql.hive.metastore.version` of these tests is lower than `2.0`, however Datanucleus 3.x seem does not support `JAVA_9` or later. Hive upgrade Datanucleus to 4.x from Hive 2.0([HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113)):

```
[info]   Cause: org.datanucleus.exceptions.NucleusException: The java type java.lang.Long (jdbc-type="", sql-type="") cant be mapped for this datastore. No mapping is available.
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.getDatastoreMappingClass(RDBMSMappingManager.java:1215)
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.createDatastoreMapping(RDBMSMappingManager.java:1378)
[info]   at org.datanucleus.store.rdbms.table.AbstractClassTable.addDatastoreId(AbstractClassTable.java:392)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.initializePK(ClassTable.java:1087)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.preInitialize(ClassTable.java:247)
```

Please note that this exclude only the tests related to the old metastore library, some other tests of `HiveSparkSubmitSuite` still fail on JDK9+.

## How was this patch tested?

manual tests:

Test with JDK 11:
```
[info] HiveExternalCatalogVersionsSuite:
[info] - backward compatibility !!! CANCELED !!! (37 milliseconds)

[info] HiveSparkSubmitSuite:
...
[info] - SPARK-8020: set sql conf in spark conf !!! CANCELED !!! (30 milliseconds)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:130)
...
[info] - SPARK-9757 Persist Parquet relation with decimal column !!! CANCELED !!! (1 millisecond)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:168)
...
[info] - SPARK-16901: set javax.jdo.option.ConnectionURL !!! CANCELED !!! (1 millisecond)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:260)
...
```

Closes #25426 from wangyum/SPARK-28703.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 20:42:06 -07:00
Stavros Kontopoulos ec84415358 [SPARK-28280][PYTHON][SQL][TESTS][FOLLOW-UP] Add UDF cases into group by clause in 'udf-group-by.sql'
## What changes were proposed in this pull request?
This PR is a followup of a fix as described in here: https://github.com/apache/spark/pull/25215#issuecomment-517659981

<details><summary>Diff comparing to 'group-by.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
index 3a5df254f2..febe47b5ba 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
 -13,26 +13,26  struct<>

 -- !query 1
-SELECT a, COUNT(b) FROM testData
+SELECT udf(a), udf(COUNT(b)) FROM testData
 -- !query 1 schema
 struct<>
 -- !query 1 output
 org.apache.spark.sql.AnalysisException
-grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.;
+grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(CAST(udf(cast(count(b) as string)) AS BIGINT) AS `CAST(udf(cast(count(b) as string)) AS BIGINT)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.;

 -- !query 2
-SELECT COUNT(a), COUNT(b) FROM testData
+SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData
 -- !query 2 schema
-struct<count(a):bigint,count(b):bigint>
+struct<count(CAST(udf(cast(a as string)) AS INT)):bigint,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 2 output
 7	7

 -- !query 3
-SELECT a, COUNT(b) FROM testData GROUP BY a
+SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a
 -- !query 3 schema
-struct<a:int,count(b):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 3 output
 1	2
 2	2
 -41,7 +41,7  NULL	1

 -- !query 4
-SELECT a, COUNT(b) FROM testData GROUP BY b
+SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b
 -- !query 4 schema
 struct<>
 -- !query 4 output
 -50,9 +50,9  expression 'testdata.`a`' is neither present in the group by, nor is it an aggre

 -- !query 5
-SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a
+SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a)
 -- !query 5 schema
-struct<count(a):bigint,count(b):bigint>
+struct<count(CAST(udf(cast(a as string)) AS INT)):bigint,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 5 output
 0	1
 2	2
 -61,15 +61,15  struct<count(a):bigint,count(b):bigint>

 -- !query 6
-SELECT 'foo', COUNT(a) FROM testData GROUP BY 1
+SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1
 -- !query 6 schema
-struct<foo:string,count(a):bigint>
+struct<foo:string,count(CAST(udf(cast(a as string)) AS INT)):bigint>
 -- !query 6 output
 foo	7

 -- !query 7
-SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 7 schema
 struct<foo:string>
 -- !query 7 output
 -77,25 +77,25  struct<foo:string>

 -- !query 8
-SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 8 schema
-struct<foo:string,approx_count_distinct(a):bigint>
+struct<foo:string,CAST(udf(cast(approx_count_distinct(cast(udf(cast(a as string)) as int), 0.05, 0, 0) as string)) AS BIGINT):bigint>
 -- !query 8 output

 -- !query 9
-SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 9 schema
-struct<foo:string,max(named_struct(a, a)):struct<a:int>>
+struct<foo:string,max(named_struct(col1, CAST(udf(cast(a as string)) AS INT))):struct<col1:int>>
 -- !query 9 output

 -- !query 10
-SELECT a + b, COUNT(b) FROM testData GROUP BY a + b
+SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b
 -- !query 10 schema
-struct<(a + b):int,count(b):bigint>
+struct<CAST(udf(cast((a + b) as string)) AS INT):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 10 output
 2	1
 3	2
 -105,7 +105,7  NULL	1

 -- !query 11
-SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1
+SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1
 -- !query 11 schema
 struct<>
 -- !query 11 output
 -114,9 +114,9  expression 'testdata.`a`' is neither present in the group by, nor is it an aggre

 -- !query 12
-SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1
+SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1)
 -- !query 12 schema
-struct<((a + 1) + 1):int,count(b):bigint>
+struct<(CAST(udf(cast((a + 1) as string)) AS INT) + 1):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 12 output
 3	2
 4	2
 -125,26 +125,26  NULL	1

 -- !query 13
-SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a)
+SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a))
 FROM testData
 -- !query 13 schema
-struct<skewness(CAST(a AS DOUBLE)):double,kurtosis(CAST(a AS DOUBLE)):double,min(a):int,max(a):int,avg(a):double,var_samp(CAST(a AS DOUBLE)):double,stddev_samp(CAST(a AS DOUBLE)):double,sum(a):bigint,count(a):bigint>
+struct<skewness(CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double,CAST(udf(cast(kurtosis(cast(a as double)) as string)) AS DOUBLE):double,CAST(udf(cast(min(a) as string)) AS INT):int,max(CAST(udf(cast(a as string)) AS INT)):int,CAST(udf(cast(avg(cast(cast(udf(cast(a as string)) as int) as bigint)) as string)) AS DOUBLE):double,CAST(udf(cast(var_samp(cast(a as double)) as string)) AS DOUBLE):double,stddev_samp(CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double,CAST(udf(cast(sum(cast(a as bigint)) as string)) AS BIGINT):bigint,CAST(udf(cast(count(a) as string)) AS BIGINT):bigint>
 -- !query 13 output
 -0.2723801058145729	-1.5069204152249134	1	3	2.142857142857143	0.8095238095238094	0.8997354108424372	15	7

 -- !query 14
-SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a
+SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a)
 -- !query 14 schema
-struct<count(DISTINCT b):bigint,count(DISTINCT b, c):bigint>
+struct<count(DISTINCT CAST(udf(cast(b as string)) AS INT)):bigint,CAST(udf(cast(count(distinct b, c) as string)) AS BIGINT):bigint>
 -- !query 14 output
 1	1

 -- !query 15
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k
+SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k
 -- !query 15 schema
-struct<k:int,count(b):bigint>
+struct<k:int,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 15 output
 1	2
 2	2
 -153,21 +153,21  NULL	1

 -- !query 16
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1
+SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1
 -- !query 16 schema
-struct<k:int,count(b):bigint>
+struct<k:int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 16 output
 2	2
 3	2

 -- !query 17
-SELECT COUNT(b) AS k FROM testData GROUP BY k
+SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k
 -- !query 17 schema
 struct<>
 -- !query 17 output
 org.apache.spark.sql.AnalysisException
-aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`);
+aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT);

 -- !query 18
 -180,7 +180,7  struct<>

 -- !query 19
-SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a
+SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a)
 -- !query 19 schema
 struct<>
 -- !query 19 output
 -197,32 +197,32  spark.sql.groupByAliases	false

 -- !query 21
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k
+SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k
 -- !query 21 schema
 struct<>
 -- !query 21 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47
+cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57

 -- !query 22
-SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a
+SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a)
 -- !query 22 schema
-struct<a:int,count(1):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,count(CAST(udf(cast(1 as string)) AS INT)):bigint>
 -- !query 22 output

 -- !query 23
-SELECT COUNT(1) FROM testData WHERE false
+SELECT udf(COUNT(1)) FROM testData WHERE false
 -- !query 23 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 23 output
 0

 -- !query 24
-SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t
+SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t
 -- !query 24 schema
 struct<1:int>
 -- !query 24 output
 -232,7 +232,7  struct<1:int>
 -- !query 25
 SELECT 1 from (
   SELECT 1 AS z,
-  MIN(a.x)
+  udf(MIN(a.x))
   FROM (select 1 as x) a
   WHERE false
 ) b
 -244,32 +244,32  struct<1:int>

 -- !query 26
-SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*)
+SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*)
   FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y)
 -- !query 26 schema
-struct<corr(DISTINCT CAST(x AS DOUBLE), CAST(y AS DOUBLE)):double,corr(DISTINCT CAST(y AS DOUBLE), CAST(x AS DOUBLE)):double,count(1):bigint>
+struct<corr(DISTINCT CAST(x AS DOUBLE), CAST(y AS DOUBLE)):double,CAST(udf(cast(corr(distinct cast(y as double), cast(x as double)) as string)) AS DOUBLE):double,count(1):bigint>
 -- !query 26 output
 1.0	1.0	3

 -- !query 27
-SELECT 1 FROM range(10) HAVING true
+SELECT udf(1) FROM range(10) HAVING true
 -- !query 27 schema
-struct<1:int>
+struct<CAST(udf(cast(1 as string)) AS INT):int>
 -- !query 27 output
 1

 -- !query 28
-SELECT 1 FROM range(10) HAVING MAX(id) > 0
+SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0
 -- !query 28 schema
-struct<1:int>
+struct<CAST(udf(cast(cast(udf(cast(1 as string)) as int) as string)) AS INT):int>
 -- !query 28 output
 1

 -- !query 29
-SELECT id FROM range(10) HAVING id > 0
+SELECT udf(id) FROM range(10) HAVING id > 0
 -- !query 29 schema
 struct<>
 -- !query 29 output
 -291,33 +291,33  struct<>

 -- !query 31
-SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0
+SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0
 -- !query 31 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(every(v) as string)) AS BOOLEAN):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 31 output
 NULL	NULL	NULL

 -- !query 32
-SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4
+SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4
 -- !query 32 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(every(cast(udf(cast(v as string)) as boolean)) as string)) AS BOOLEAN):boolean,some(v):boolean,any(v):boolean>
 -- !query 32 output
 NULL	NULL	NULL

 -- !query 33
-SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5
+SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5
 -- !query 33 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<every(v):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 33 output
 false	true	true

 -- !query 34
-SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k
+SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k)
 -- !query 34 schema
-struct<k:int,every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every(v):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 34 output
 1	false	true	true
 2	true	true	true
 -327,9 +327,9  struct<k:int,every(v):boolean,some(v):boolean,any(v):boolean>

 -- !query 35
-SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false
+SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false
 -- !query 35 schema
-struct<k:int,every(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every(v):boolean>
 -- !query 35 output
 1	false
 3	false
 -337,77 +337,77  struct<k:int,every(v):boolean>

 -- !query 36
-SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL
+SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL
 -- !query 36 schema
-struct<k:int,every(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(every(v) as string)) AS BOOLEAN):boolean>
 -- !query 36 output
 4	NULL

 -- !query 37
-SELECT k,
-       Every(v) AS every
+SELECT udf(k),
+       udf(Every(v)) AS every
 FROM   test_agg
 WHERE  k = 2
        AND v IN (SELECT Any(v)
                  FROM   test_agg
                  WHERE  k = 1)
-GROUP  BY k
+GROUP  BY udf(k)
 -- !query 37 schema
-struct<k:int,every:boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every:boolean>
 -- !query 37 output
 2	true

 -- !query 38
-SELECT k,
+SELECT udf(udf(k)),
        Every(v) AS every
 FROM   test_agg
 WHERE  k = 2
        AND v IN (SELECT Every(v)
                  FROM   test_agg
                  WHERE  k = 1)
-GROUP  BY k
+GROUP  BY udf(udf(k))
 -- !query 38 schema
-struct<k:int,every:boolean>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,every:boolean>
 -- !query 38 output

 -- !query 39
-SELECT every(1)
+SELECT every(udf(1))
 -- !query 39 schema
 struct<>
 -- !query 39 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7
+cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7

 -- !query 40
-SELECT some(1S)
+SELECT some(udf(1S))
 -- !query 40 schema
 struct<>
 -- !query 40 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7
+cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7

 -- !query 41
-SELECT any(1L)
+SELECT any(udf(1L))
 -- !query 41 schema
 struct<>
 -- !query 41 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7
+cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7

 -- !query 42
-SELECT every("true")
+SELECT udf(every("true"))
 -- !query 42 schema
 struct<>
 -- !query 42 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7
+cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11

 -- !query 43
 -428,9 +428,9  struct<k:int,v:boolean,every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST

 -- !query 44
-SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
+SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
 -- !query 44 schema
-struct<k:int,v:boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
+struct<k:int,CAST(udf(cast(cast(udf(cast(v as string)) as boolean) as string)) AS BOOLEAN):boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
 -- !query 44 output
 1	false	false
 1	true	true
 -445,9 +445,9  struct<k:int,v:boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST R

 -- !query 45
-SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
+SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
 -- !query 45 schema
-struct<k:int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
 -- !query 45 output
 1	false	false
 1	true	true
 -462,17 +462,17  struct<k:int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RA

 -- !query 46
-SELECT count(*) FROM test_agg HAVING count(*) > 1L
+SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L
 -- !query 46 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 46 output
 10

 -- !query 47
-SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true
+SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true
 -- !query 47 schema
-struct<k:int,max(v):boolean>
+struct<k:int,CAST(udf(cast(max(v) as string)) AS BOOLEAN):boolean>
 -- !query 47 output
 1	true
 2	true
 -480,7 +480,7  struct<k:int,max(v):boolean>

 -- !query 48
-SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L
+SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L
 -- !query 48 schema
 struct<cnt:bigint>
 -- !query 48 output
 -488,7 +488,7  struct<cnt:bigint>

 -- !query 49
-SELECT count(*) FROM test_agg WHERE count(*) > 1L
+SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L
 -- !query 49 schema
 struct<>
 -- !query 49 output
 -500,7 +500,7  Invalid expressions: [count(1)];

 -- !query 50
-SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L
+SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L
 -- !query 50 schema
 struct<>
 -- !query 50 output
 -512,7 +512,7  Invalid expressions: [count(1)];

 -- !query 51
-SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1
+SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1
 -- !query 51 schema
 struct<>
 -- !query 51 output

```

</p>
</details>

## How was this patch tested?
Tested as instructed in SPARK-27921.

Closes #25360 from skonto/group-by-followup.

Authored-by: Stavros Kontopoulos <st.kontopoulos@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-13 10:06:32 +09:00
s71955 163f4a45df [SPARK-26969][SQL] Using ODBC client not able to see the query data when column datatype is decimal
## What changes were proposed in this pull request?
While processing the Rowdata in the server side ColumnValue BigDecimal type value processed by server has to converted to the HiveDecmal data type for successful processing of query using Hive ODBC client.As per current logic  corresponding to the Decimal column datatype, the Spark server uses BigDecimal, and the ODBC client uses HiveDecimal. If the data type does not match, the client fail to parse

Since this handing was missing the query executed in Hive ODBC client wont return or provides result to the user even though the decimal type column value data present.

## How was this patch tested?

Manual test report and impact assessment is done using existing test-cases

Before fix
![decimal_odbc](https://user-images.githubusercontent.com/12999161/53440179-e74a7f00-3a29-11e9-93db-83f2ae37ef16.PNG)

After Fix
![hive_odbc](https://user-images.githubusercontent.com/12999161/53679519-70e0a200-3cf3-11e9-9437-9c27d2e5056d.PNG)

Closes #23899 from sujith71955/master_decimalissue.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 15:47:59 -07:00
Maxim Gekk 6964128e25 [SPARK-28017][SPARK-28656][SQL][FOLLOW-UP] Restore comments in date.sql
## What changes were proposed in this pull request?

Restored comments in `date.sql` removed by 924d794a6f and 997d153e54 . The comments was introduced by 51379b731d .

## How was this patch tested?

By re-running `date.sql` via:
```shell
$ build/sbt "sql/test-only *SQLQueryTestSuite -- -z date.sql"
```

Closes #25422 from MaxGekk/sql-comments-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 11:19:19 -07:00
Yuming Wang e5f4a106db [SPARK-28688][SQL][TEST] Skip VersionsSuite.read hive materialized view test for HMS 3.0+ on JDK9+
## What changes were proposed in this pull request?

This PR makes it skip test `read hive materialized view` since Hive 3.0 in `VersionsSuite.scala` on JDK 11 because [HIVE-19383](https://issues.apache.org/jira/browse/HIVE-19383) added [ArrayList$SubList](ae4df62795/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java (L383)) which is incompatible with JDK 11:
```java
java.lang.RuntimeException: java.lang.NoSuchFieldException: parentOffset
	at org.apache.hadoop.hive.ql.exec.SerializationUtilities$ArrayListSubListSerializer.<init>(SerializationUtilities.java:389)
	at org.apache.hadoop.hive.ql.exec.SerializationUtilities$1.create(SerializationUtilities.java:235)
...
```
![image](https://issues.apache.org/jira/secure/attachment/12977250/12977250_screenshot-2.png)
![image](https://issues.apache.org/jira/secure/attachment/12977249/12977249_screenshot-1.png)

## How was this patch tested?

manual tests
**Test on JDK 11**:
```
...
[info] - 2.3: sql read hive materialized view (1 second, 253 milliseconds)
...
[info] - 3.0: sql read hive materialized view !!! CANCELED !!! (31 milliseconds)
[info]   "[3.0]" did not equal "[2.3]", and org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (VersionsSuite.scala:624)
...
[info] - 3.1: sql read hive materialized view !!! CANCELED !!! (0 milliseconds)
[info]   "[3.1]" did not equal "[2.3]", and org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (VersionsSuite.scala:624)
...
```

**Test on JDK 1.8**:
```
...
[info] - 2.3: sql read hive materialized view (1 second, 444 milliseconds)
...
[info] - 3.0: sql read hive materialized view (3 seconds, 100 milliseconds)
...
[info] - 3.1: sql read hive materialized view (2 seconds, 941 milliseconds)
...
```

Closes #25414 from wangyum/SPARK-28688.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 03:37:10 -07:00
Yuming Wang 6c06eea411 [SPARK-28686][SQL][TEST] Move udf_radians from HiveCompatibilitySuite to HiveQuerySuite
## What changes were proposed in this pull request?

This PR moves `udf_radians` from `HiveCompatibilitySuite` to `HiveQuerySuite` to make it easy to test with JDK 11 because it returns different value from JDK 9:
```java
public class TestRadians {
  public static void main(String[] args) {
    System.out.println(java.lang.Math.toRadians(57.2958));
  }
}
```
```sh
[rootspark-3267648 ~]# javac TestRadians.java
[rootspark-3267648 ~]# /usr/lib/jdk-9.0.4+11/bin/java TestRadians
1.0000003575641672
[rootspark-3267648 ~]# /usr/lib/jdk-11.0.3/bin/java TestRadians
1.0000003575641672
[rootspark-3267648 ~]# /usr/lib/jdk8u222-b10/bin/java TestRadians
1.000000357564167
```

## How was this patch tested?

manual tests

Closes #25417 from wangyum/SPARK-28686.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 02:24:48 -07:00
Yuming Wang 58cc0df59e [SPARK-28685][SQL][TEST] Test HMS 2.0.0+ in VersionsSuite/HiveClientSuites on JDK 11
## What changes were proposed in this pull request?

It seems Datanucleus 3.x can not support JDK 11:
```java
[info]   Cause: org.datanucleus.exceptions.NucleusException: The java type java.lang.Long (jdbc-type="", sql-type="") cant be mapped for this datastore. No mapping is available.
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.getDatastoreMappingClass(RDBMSMappingManager.java:1215)
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.createDatastoreMapping(RDBMSMappingManager.java:1378)
[info]   at org.datanucleus.store.rdbms.table.AbstractClassTable.addDatastoreId(AbstractClassTable.java:392)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.initializePK(ClassTable.java:1087)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.preInitialize(ClassTable.java:247)
```

Hive upgrade Datanucleus to 4.x from Hive 2.0([HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113)). This PR makes it skip `0.12`, `0.13`, `0.14`, `1.0`, `1.1` and `1.2` when testing with JDK 11.

Note that, this pr will not fix sql read hive materialized view. It's another issue:
```
3.0: sql read hive materialized view *** FAILED *** (1 second, 521 milliseconds)
3.1: sql read hive materialized view *** FAILED *** (1 second, 536 milliseconds)
```

## How was this patch tested?

manual tests:
```shell
export JAVA_HOME="/usr/lib/jdk-11.0.3"
build/sbt "hive/test-only *.VersionsSuite *.HiveClientSuites" -Phive -Phadoop-3.2
```

Closes #25405 from wangyum/SPARK-28685.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-10 17:01:15 -07:00
Yuming Wang 47af8925b6 [SPARK-28675][SQL] Remove maskCredentials and use redactOptions
## What changes were proposed in this pull request?

This PR replaces `CatalogUtils.maskCredentials` with `SQLConf.get.redactOptions` to match other redacts.

## How was this patch tested?

unit test and manual tests:
Before this PR:
```sql
spark-sql> DESC EXTENDED test_spark_28675;
id	int	NULL

# Detailed Table Information
Database	default
Table	test_spark_28675
Owner	root
Created Time	Fri Aug 09 08:23:17 GMT-07:00 2019
Last Access	Wed Dec 31 17:00:00 GMT-07:00 1969
Created By	Spark 3.0.0-SNAPSHOT
Type	MANAGED
Provider	org.apache.spark.sql.jdbc
Location	file:/user/hive/warehouse/test_spark_28675
Serde Library	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat	org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties	[url=###, driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]

spark-sql> SHOW TABLE EXTENDED LIKE 'test_spark_28675';
default	test_spark_28675	false	Database: default
Table: test_spark_28675
Owner: root
Created Time: Fri Aug 09 08:23:17 GMT-07:00 2019
Last Access: Wed Dec 31 17:00:00 GMT-07:00 1969
Created By: Spark 3.0.0-SNAPSHOT
Type: MANAGED
Provider: org.apache.spark.sql.jdbc
Location: file:/user/hive/warehouse/test_spark_28675
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties: [url=###, driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]
Schema: root
 |-- id: integer (nullable = true)

```

After this PR:
```sql
spark-sql> DESC EXTENDED test_spark_28675;
id	int	NULL

# Detailed Table Information
Database	default
Table	test_spark_28675
Owner	root
Created Time	Fri Aug 09 08:19:49 GMT-07:00 2019
Last Access	Wed Dec 31 17:00:00 GMT-07:00 1969
Created By	Spark 3.0.0-SNAPSHOT
Type	MANAGED
Provider	org.apache.spark.sql.jdbc
Location	file:/user/hive/warehouse/test_spark_28675
Serde Library	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat	org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties	[url=*********(redacted), driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]

spark-sql> SHOW TABLE EXTENDED LIKE 'test_spark_28675';
default	test_spark_28675	false	Database: default
Table: test_spark_28675
Owner: root
Created Time: Fri Aug 09 08:19:49 GMT-07:00 2019
Last Access: Wed Dec 31 17:00:00 GMT-07:00 1969
Created By: Spark 3.0.0-SNAPSHOT
Type: MANAGED
Provider: org.apache.spark.sql.jdbc
Location: file:/user/hive/warehouse/test_spark_28675
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties: [url=*********(redacted), driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]
Schema: root
 |-- id: integer (nullable = true)
```

Closes #25395 from wangyum/SPARK-28675.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-10 16:45:59 -07:00
younggyu chun 8535df7261 [MINOR] Fix typos in comments and replace an explicit type with <>
## What changes were proposed in this pull request?
This PR fixed typos in comments and replace the explicit type with '<>' for Java 8+.

## How was this patch tested?
Manually tested.

Closes #25338 from younggyuchun/younggyu.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-10 16:47:11 -05:00
Maxim Gekk 924d794a6f [SPARK-28656][SQL] Support millennium, century and decade at extract()
## What changes were proposed in this pull request?

In the PR, I propose new expressions `Millennium`, `Century` and `Decade`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT):

1. `millennium` - the current millennium for given date (or a timestamp implicitly casted to a date). For example, years in the 1900s are in the second millennium. The third millennium started _January 1, 2001_.
2. `century` - the current millennium for given date (or timestamp). The first century starts at 0001-01-01 AD.
3. `decade` - the current decade for given date (or timestamp). Actually, this is the year field divided by 10.

Here are examples:
```sql
spark-sql> SELECT EXTRACT(MILLENNIUM FROM DATE '1981-01-19');
2
spark-sql> SELECT EXTRACT(CENTURY FROM DATE '1981-01-19');
20
spark-sql> SELECT EXTRACT(DECADE FROM DATE '1981-01-19');
198
```

## How was this patch tested?

Added new tests to `DateExpressionsSuite` and uncommented existing tests in `pgSQL/date.sql`.

Closes #25388 from MaxGekk/extract-ext2.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-09 11:18:50 -07:00
gengjiaan 5159876415 [SPARK-28077][SQL][TEST][FOLLOW-UP] Enable Overlay function tests
## What changes were proposed in this pull request?

This PR is a follow-up to https://github.com/apache/spark/pull/24918

## How was this patch tested?

Pass the Jenkins with the newly update test files.

Closes #25393 from beliefer/enable-overlay-tests.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-09 19:05:41 +09:00
Shixiong Zhu 5bb69945e4 [SPARK-28651][SS] Force the schema of Streaming file source to be nullable
## What changes were proposed in this pull request?

Right now, batch DataFrame always changes the schema to nullable automatically (See this line: 325bc8e9c6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L399)). But streaming file source is missing this.

This PR updates the streaming file source schema to force it be nullable. I also added a flag `spark.sql.streaming.fileSource.schema.forceNullable` to disable this change since some users may rely on the old behavior.

## How was this patch tested?

The new unit test.

Closes #25382 from zsxwing/SPARK-28651.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-09 18:54:55 +09:00
Burak Yavuz 5368eaa2fc [SPARK-28565][SQL] DataFrameWriter saveAsTable support for V2 catalogs
## What changes were proposed in this pull request?

Adds support for V2 catalogs and the V2SessionCatalog for V2 tables for saveAsTable.
If the table can resolve through the V2SessionCatalog, we use SaveMode for datasource v1 for backwards compatibility to select the code path we're going to hit.

Depending on the SaveMode:
 - SaveMode.Append:
     a) If table exists: Use AppendData.byName
     b) If table doesn't exist, use CTAS (ignoreIfExists = false)
 - SaveMode.Overwrite: Use RTAS (orCreate = true)
 - SaveMode.Ignore: Use CTAS (ignoreIfExists = true)
 - SaveMode.ErrorIfExists: Use CTAS (ignoreIfExists = false)

## How was this patch tested?

Unit tests in DataSourceV2DataFrameSuite

Closes #25330 from brkyvz/saveAsTable.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-08 22:30:00 -07:00
Maxim Gekk 997d153e54 [SPARK-28017][SQL] Support additional levels of truncations by DATE_TRUNC/TRUNC
## What changes were proposed in this pull request?

I propose new levels of truncations for the `date_trunc()` and `trunc()` functions:
1. `MICROSECOND` and `MILLISECOND` truncate values of the `TIMESTAMP` type to microsecond and millisecond precision.
2. `DECADE`, `CENTURY` and `MILLENNIUM` truncate dates/timestamps to lowest date of current decade/century/millennium.

Also the `WEEK` and `QUARTER` levels have been supported by the `trunc()` function.

The function is implemented similarly to `date_trunc` in PostgreSQL: https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-TRUNC to maintain feature parity with it.

Here are examples of `TRUNC`:
```sql
spark-sql> SELECT TRUNC('2015-10-27', 'DECADE');
2010-01-01
spark-sql> set spark.sql.datetime.java8API.enabled=true;
spark.sql.datetime.java8API.enabled	true
spark-sql> SELECT TRUNC('1999-10-27', 'millennium');
1001-01-01
```
Examples of `DATE_TRUNC`:
```sql
spark-sql> SELECT DATE_TRUNC('CENTURY', '2015-03-05T09:32:05.123456');
2001-01-01T00:00:00Z
```

## How was this patch tested?

Added new tests to `DateTimeUtilsSuite`, `DateExpressionsSuite` and `DateFunctionsSuite`, and uncommented existing tests in `pgSQL/date.sql`.

Closes #25336 from MaxGekk/date_truct-ext.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-09 12:29:44 +08:00
Burak Yavuz c80430f5c9 [SPARK-28572][SQL] Simple analyzer checks for v2 table creation code paths
## What changes were proposed in this pull request?

Adds checks around:
 - The existence of transforms in the table schema (even in nested fields)
 - Duplications of transforms
 - Case sensitivity checks around column names
in the V2 table creation code paths.

## How was this patch tested?

Unit tests.

Closes #25305 from brkyvz/v2CreateTable.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-09 12:04:28 +08:00
Yuming Wang 2580c1bfe2 [SPARK-28660][SQL][TEST] Port AGGREGATES.sql [Part 4]
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/aggregates.out#L1615-L2289

When porting the test cases, found five PostgreSQL specific features that do not exist in Spark SQL:

[SPARK-27980](https://issues.apache.org/jira/browse/SPARK-27980): Ordered-Set Aggregate Functions
[SPARK-28661](https://issues.apache.org/jira/browse/SPARK-28661): Hypothetical-Set Aggregate Functions
[SPARK-28382](https://issues.apache.org/jira/browse/SPARK-28382): Array Functions: unnest
[SPARK-28663](https://issues.apache.org/jira/browse/SPARK-28663): Aggregate Functions for Statistics
[SPARK-28664](https://issues.apache.org/jira/browse/SPARK-28664): ORDER BY in aggregate function
[SPARK-28669](https://issues.apache.org/jira/browse/SPARK-28669): System Information Functions

## How was this patch tested?

N/A

Closes #25392 from wangyum/SPARK-28660.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-08 16:39:32 -07:00
Yuming Wang d19a56f9db [SPARK-28642][SQL] Hide credentials in SHOW CREATE TABLE
## What changes were proposed in this pull request?

[SPARK-17783](https://issues.apache.org/jira/browse/SPARK-17783) hided Credentials in `CREATE` and `DESC FORMATTED/EXTENDED` a PERSISTENT/TEMP Table for JDBC. But `SHOW
 CREATE TABLE` exposed the credentials:
```sql
spark-sql> show create table mysql_federated_sample;
CREATE TABLE `mysql_federated_sample` (`TBL_ID` BIGINT, `CREATE_TIME` INT, `DB_ID` BIGINT, `LAST_ACCESS_TIME` INT, `OWNER` STRING, `RETENTION` INT, `SD_ID` BIGINT, `TBL_NAME` STRING, `TBL_TYPE` STRING, `VIEW_EXPANDED_TEXT` STRING, `VIEW_ORIGINAL_TEXT` STRING, `IS_REWRITE_ENABLED` BOOLEAN)
USING org.apache.spark.sql.jdbc
OPTIONS (
  `url` 'jdbc:mysql://localhost/hive?user=root&password=mypasswd',
  `driver` 'com.mysql.jdbc.Driver',
  `dbtable` 'TBLS'
)
```

This pr fix this issue.

## How was this patch tested?

unit tests and manual tests:
```sql
spark-sql> show create table  mysql_federated_sample;
CREATE TABLE `mysql_federated_sample` (`TBL_ID` BIGINT, `CREATE_TIME` INT, `DB_ID` BIGINT, `LAST_ACCESS_TIME` INT, `OWNER` STRING, `RETENTION` INT, `SD_ID` BIGINT, `TBL_NAME` STRING, `TBL_TYPE` STRING, `VIEW_EXPANDED_TEXT` STRING, `VIEW_ORIGINAL_TEXT` STRING, `IS_REWRITE_ENABLED` BOOLEAN)
USING org.apache.spark.sql.jdbc
OPTIONS (
  `url` '*********(redacted)',
  `driver` 'com.mysql.jdbc.Driver',
  `dbtable` 'TBLS'
)
```

Closes #25375 from wangyum/SPARK-28642.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-08 16:24:43 -07:00
HyukjinKwon 8c0dc38640 [SPARK-28654][SQL] Move "Extract Python UDFs" to the last in optimizer
## What changes were proposed in this pull request?

Plans after "Extract Python UDFs" are very flaky and error-prone to other rules.

For instance, if we add some rules, for instance, `PushDownPredicates` in `postHocOptimizationBatches`, the test in `BatchEvalPythonExecSuite` fails:

```scala
test("Python UDF refers to the attributes from more than one child") {
  val df = Seq(("Hello", 4)).toDF("a", "b")
  val df2 = Seq(("Hello", 4)).toDF("c", "d")
  val joinDF = df.crossJoin(df2).where("dummyPythonUDF(a, c) == dummyPythonUDF(d, c)")
  val qualifiedPlanNodes = joinDF.queryExecution.executedPlan.collect {
    case b: BatchEvalPythonExec => b
  }
  assert(qualifiedPlanNodes.size == 1)
}
```

```
Invalid PythonUDF dummyUDF(a#63, c#74), requires attributes from more than one child.
```

This is because Python UDF extraction optimization is rolled back as below:

```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.PushDownPredicates ===
!Filter (dummyUDF(a#7, c#18) = dummyUDF(d#19, c#18))   Join Cross, (dummyUDF(a#7, c#18) = dummyUDF(d#19, c#18))
!+- Join Cross                                         :- Project [_1#2 AS a#7, _2#3 AS b#8]
!   :- Project [_1#2 AS a#7, _2#3 AS b#8]              :  +- LocalRelation [_1#2, _2#3]
!   :  +- LocalRelation [_1#2, _2#3]                   +- Project [_1#13 AS c#18, _2#14 AS d#19]
!   +- Project [_1#13 AS c#18, _2#14 AS d#19]             +- LocalRelation [_1#13, _2#14]
!      +- LocalRelation [_1#13, _2#14]
```

Seems we should do Python UDFs cases at the last even after post hoc rules.

Note that this actually rather follows the way in previous versions when those were in physical plans (see SPARK-24721 and SPARK-12981). Those optimization rules were supposed to be placed at the end.

Note that I intentionally didn't move `ExperimentalMethods` (`spark.experimental.extraStrategies`). This is an explicit experimental API and I wanted to just-in-case workaround after this change for now.

## How was this patch tested?

Existing tests should cover.

Closes #25386 from HyukjinKwon/SPARK-28654.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-08 20:21:07 +08:00
Yuming Wang 1941d35d1e [SPARK-28644][SQL] Port HIVE-10646: ColumnValue does not handle NULL_TYPE
## What changes were proposed in this pull request?

This PR port [HIVE-10646](https://issues.apache.org/jira/browse/HIVE-10646) to fix Hive 0.12's JDBC client can not handle `NULL_TYPE`:
```sql
Connected to: Hive (version 3.0.0-SNAPSHOT)
Driver: Hive (version 0.12.0)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 0.12.0 by Apache Hive
0: jdbc:hive2://localhost:10000> select null;
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:346)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:423)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:405)
```

Server log:
```
19/08/07 09:34:07 ERROR TThreadPoolServer: Error occurred during processing of message.
java.lang.NullPointerException
	at org.apache.hive.service.cli.thrift.TRow$TRowStandardScheme.write(TRow.java:388)
	at org.apache.hive.service.cli.thrift.TRow$TRowStandardScheme.write(TRow.java:338)
	at org.apache.hive.service.cli.thrift.TRow.write(TRow.java:288)
	at org.apache.hive.service.cli.thrift.TRowSet$TRowSetStandardScheme.write(TRowSet.java:605)
	at org.apache.hive.service.cli.thrift.TRowSet$TRowSetStandardScheme.write(TRowSet.java:525)
	at org.apache.hive.service.cli.thrift.TRowSet.write(TRowSet.java:455)
	at org.apache.hive.service.cli.thrift.TFetchResultsResp$TFetchResultsRespStandardScheme.write(TFetchResultsResp.java:550)
	at org.apache.hive.service.cli.thrift.TFetchResultsResp$TFetchResultsRespStandardScheme.write(TFetchResultsResp.java:486)
	at org.apache.hive.service.cli.thrift.TFetchResultsResp.write(TFetchResultsResp.java:412)
	at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result$FetchResults_resultStandardScheme.write(TCLIService.java:13192)
	at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result$FetchResults_resultStandardScheme.write(TCLIService.java:13156)
	at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result.write(TCLIService.java:13107)
	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:58)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:819)
```

## How was this patch tested?

unit tests

Closes #25378 from wangyum/SPARK-28644.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 17:28:10 +09:00
Yuming Wang c4acfe7761 [SPARK-28474][SQL] Hive 0.12 JDBC client can not handle binary type
## What changes were proposed in this pull request?

This PR fix Hive 0.12 JDBC client can not handle binary type:
```sql
Connected to: Hive (version 3.0.0-SNAPSHOT)
Driver: Hive (version 0.12.0)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 0.12.0 by Apache Hive
0: jdbc:hive2://localhost:10000> SELECT cast('ABC' as binary);
Error: java.lang.ClassCastException: [B incompatible with java.lang.String (state=,code=0)
```

Server log:
```
19/08/07 10:10:04 WARN ThriftCLIService: Error fetching results:
java.lang.RuntimeException: java.lang.ClassCastException: [B incompatible with java.lang.String
	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:83)
	at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36)
	at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63)
	at java.security.AccessController.doPrivileged(AccessController.java:770)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1746)
	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59)
	at com.sun.proxy.$Proxy26.fetchResults(Unknown Source)
	at org.apache.hive.service.cli.CLIService.fetchResults(CLIService.java:455)
	at org.apache.hive.service.cli.thrift.ThriftCLIService.FetchResults(ThriftCLIService.java:621)
	at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1553)
	at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1538)
	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:38)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:819)
Caused by: java.lang.ClassCastException: [B incompatible with java.lang.String
	at org.apache.hive.service.cli.ColumnValue.toTColumnValue(ColumnValue.java:198)
	at org.apache.hive.service.cli.RowBasedSet.addRow(RowBasedSet.java:60)
	at org.apache.hive.service.cli.RowBasedSet.addRow(RowBasedSet.java:32)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.$anonfun$getNextRowSet$1(SparkExecuteStatementOperation.scala:151)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$Lambda$1923.000000009113BFE0.apply(Unknown Source)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withSchedulerPool(SparkExecuteStatementOperation.scala:299)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.getNextRowSet(SparkExecuteStatementOperation.scala:113)
	at org.apache.hive.service.cli.operation.OperationManager.getOperationNextRowSet(OperationManager.java:220)
	at org.apache.hive.service.cli.session.HiveSessionImpl.fetchResults(HiveSessionImpl.java:785)
	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.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78)
	... 18 more
```

## How was this patch tested?

unit tests

Closes #25379 from wangyum/SPARK-28474.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 17:01:25 +09:00
Yishuang Lu e58dd4af60 [MINOR][DOC] Fix a typo 'lister' -> 'listener'
## What changes were proposed in this pull request?

Fix the typo in java doc.

## How was this patch tested?

N/A

Signed-off-by: Yishuang Lu <luystugmail.com>

Closes #25377 from lys0716/dev.

Authored-by: Yishuang Lu <luystu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 11:12:18 +09:00
Yuming Wang 3586cdd24d [SPARK-28395][FOLLOW-UP][SQL] Make spark.sql.function.preferIntegralDivision internal
## What changes were proposed in this pull request?

This PR makes `spark.sql.function.preferIntegralDivision` to internal configuration because it is only used for PostgreSQL test cases.

More details:
https://github.com/apache/spark/pull/25158#discussion_r309764541

## How was this patch tested?

N/A

Closes #25376 from wangyum/SPARK-28395-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 10:42:24 +09:00
Yuming Wang eeaf1851b2 [SPARK-28617][SQL][TEST] Fix misplacement when comment is at the end of the query
## What changes were proposed in this pull request?

This PR fixes the issue of misplacement when the comment at the end of the query. Example:
Comment for ` SELECT date '5874898-01-01'`:
2d74f14d74/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql (L200)
But the golden file is:
a5a5da78cf/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out (L484-L507)

After this PR:
eeb7405ad0/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out (L482-L501)

## How was this patch tested?

N/A

Closes #25357 from wangyum/SPARK-28617.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-07 16:45:23 -07:00
Gengliang Wang c88df2ccf6 [SPARK-28331][SQL] Catalogs.load() should be able to load built-in catalogs
## What changes were proposed in this pull request?

In `Catalogs.load`, the `pluginClassName` in the following code
```
String pluginClassName = conf.getConfString("spark.sql.catalog." + name, null);
```
is always null for built-in catalogs, e.g there is a SQLConf entry `spark.sql.catalog.session`.

This is because of https://github.com/apache/spark/pull/18852: SQLConf.conf.getConfString(key, null) always returns null.

## How was this patch tested?

Apply code changes of https://github.com/apache/spark/pull/24768 and tried loading session catalog.

Closes #25094 from gengliangwang/fixCatalogLoad.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-07 16:14:34 -07:00
Marco Gaido 8617bf6ff8 [SPARK-28470][SQL] Cast to decimal throws ArithmeticException on overflow
## What changes were proposed in this pull request?

The flag `spark.sql.decimalOperations.nullOnOverflow` is not honored by the `Cast` operator. This means that a casting which causes an overflow currently returns `null`.

The PR makes `Cast` respecting that flag, ie. when it is turned to false and a decimal overflow occurs, an exception id thrown.

## How was this patch tested?

Added UT

Closes #25253 from mgaido91/SPARK-28470.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-08 08:10:21 +09:00
maryannxue 325bc8e9c6 [SPARK-28583][SQL] Subqueries should not call onUpdatePlan in Adaptive Query Execution
## What changes were proposed in this pull request?

Subqueries do not have their own execution id, thus when calling `AdaptiveSparkPlanExec.onUpdatePlan`, it will actually get the `QueryExecution` instance of the main query, which is wasteful and problematic. It could cause issues like stack overflow or dead locks in some circumstances.

This PR fixes this issue by making `AdaptiveSparkPlanExec` compare the `QueryExecution` object retrieved by current execution ID against the `QueryExecution` object from which this plan is created, and only update the UI when the two instances are the same.

## How was this patch tested?

Manual tests on TPC-DS queries.

Closes #25316 from maryannxue/aqe-updateplan-fix.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: herman <herman@databricks.com>
2019-08-07 22:10:17 +02:00
Yuming Wang a59fdc4b57 [SPARK-28472][SQL][TEST] Add test for thriftserver protocol versions
## What changes were proposed in this pull request?

This pr adds a test(`SparkThriftServerProtocolVersionsSuite`) to test different versions of the thrift protocol because we use different logic to handle the `RowSet`:
02c33694c8/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (L28-L40)

When adding this test cases, found three bugs:
[SPARK-26969](https://issues.apache.org/jira/browse/SPARK-26969): Using ODBC not able to see the data in table when datatype is decimal
[SPARK-28463](https://issues.apache.org/jira/browse/SPARK-28463): Thriftserver throws BigDecimal incompatible with HiveDecimal
[SPARK-28474](https://issues.apache.org/jira/browse/SPARK-28474): Lower JDBC client version(Hive 0.12) cannot read binary type

## How was this patch tested?

N/A

Closes #25228 from wangyum/SPARK-28472.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-07 08:51:58 -07:00
Wenchen Fan 469423f338 [SPARK-28595][SQL] explain should not trigger partition listing
## What changes were proposed in this pull request?

Sometimes when you explain a query, you will get stuck for a while. What's worse, you will get stuck again if you explain again.

This is caused by `FileSourceScanExec`:
1. In its `toString`, it needs to report the number of partitions it reads. This needs to query the hive metastore.
2. In its `outputOrdering`, it needs to get all the files. This needs to query the hive metastore.

This PR fixes by:
1. `toString` do not need to report the number of partitions it reads. We should report it via SQL metrics.
2. The `outputOrdering` is not very useful. We can only apply it if a) all the bucket columns are read. b) there is only one file in each bucket. This condition is really hard to meet, and even if we meet, sorting an already sorted file is pretty fast and avoiding the sort is not that useful. I think it's worth to give up this optimization so that explain don't need to get stuck.

## How was this patch tested?

existing tests

Closes #25328 from cloud-fan/ui.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-07 19:14:25 +08:00
gengjiaan 99de6a4240 [SPARK-27924][SQL][TEST][FOLLOW-UP] Enable Boolean-Predicate syntax tests
## What changes were proposed in this pull request?

This PR is a follow-up to https://github.com/apache/spark/pull/25074

## How was this patch tested?

Pass the Jenkins with the newly update test files.

Closes #25366 from beliefer/uncomment-boolean-test.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-07 00:34:49 -07:00
mcheah 44e607e921 [SPARK-28238][SQL] Implement DESCRIBE TABLE for Data Source V2 Tables
## What changes were proposed in this pull request?

Implements the `DESCRIBE TABLE` logical and physical plans for data source v2 tables.

## How was this patch tested?

Added unit tests to `DataSourceV2SQLSuite`.

Closes #25040 from mccheah/describe-table-v2.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-07 14:26:45 +08:00
WeichenXu a133175ffa [SPARK-28615][SQL][DOCS] Add a guide line for dataframe functions to say column signature function is by default
## What changes were proposed in this pull request?

Add a guide line for dataframe functions, say:
```
This function APIs usually have methods with Column signature only because it can support not only Column but also other types such as a native string. The other variants currently exist for historical reasons.
```

## How was this patch tested?

N/A

Closes #25355 from WeichenXu123/update_functions_guide2.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-07 10:39:47 +09:00
Nik Vanderhoof 9e931e787d [SPARK-27905][SQL] Add higher order function 'forall'
## What changes were proposed in this pull request?

Add's the higher order function `forall`, which tests an array to see if a predicate holds for every element.
The function is implemented in `org.apache.spark.sql.catalyst.expressions.ArrayForAll`.
The function is added to the function registry under the pretty name `forall`.

## How was this patch tested?

I've added appropriate unit tests for the new ArrayForAll expression in
`sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala`.

Also added tests for the function in `sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala`.

Not sure who is best to ask about this PR so:
 HyukjinKwon rxin gatorsmile ueshin srowen hvanhovell gatorsmile

Closes #24761 from nvander1/feature/for_all.

Lead-authored-by: Nik Vanderhoof <nikolasrvanderhoof@gmail.com>
Co-authored-by: Nik <nikolasrvanderhoof@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2019-08-06 14:25:53 -07:00
Maxim Gekk 9e3aab8b95 [SPARK-28623][SQL] Support dow, isodow and doy by extract()
## What changes were proposed in this pull request?

In the PR, I propose to use existing expressions `DayOfYear`, `WeekDay` and `DayOfWeek`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT):

1. `dow` - the day of the week as Sunday (0) to Saturday (6)
2. `isodow` - the day of the week as Monday (1) to Sunday (7)
3. `doy` - the day of the year (1 - 365/366)

Here are examples:
```sql
spark-sql> SELECT EXTRACT(DOW FROM TIMESTAMP '2001-02-16 20:38:40');
5
spark-sql> SELECT EXTRACT(ISODOW FROM TIMESTAMP '2001-02-18 20:38:40');
7
spark-sql> SELECT EXTRACT(DOY FROM TIMESTAMP '2001-02-16 20:38:40');
47
```

## How was this patch tested?

Updated `extract.sql`.

Closes #25367 from MaxGekk/extract-ext.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-06 13:39:49 -07:00
HyukjinKwon bab88c48b1 [SPARK-28622][SQL][PYTHON] Rename PullOutPythonUDFInJoinCondition to ExtractPythonUDFFromJoinCondition and move to 'Extract Python UDFs'
## What changes were proposed in this pull request?

This PR targets to rename `PullOutPythonUDFInJoinCondition` to `ExtractPythonUDFFromJoinCondition` and move to 'Extract Python UDFs' together with other Python UDF related rules.

Currently `PullOutPythonUDFInJoinCondition` rule is alone outside of other 'Extract Python UDFs' rules together.

and the name `ExtractPythonUDFFromJoinCondition` is matched to existing Python UDF extraction rules.

## How was this patch tested?

Existing tests should cover.

Closes #25358 from HyukjinKwon/move-python-join-rule.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-05 23:36:35 -07:00
Udbhav30 150dbc5dc2 [SPARK-28391][PYTHON][SQL][TESTS][FOLLOW-UP] Add UDF cases into groupby clause in 'pgSQL/select_implicit.sql'
## What changes were proposed in this pull request?
This PR adds UDF cases into group by clause in 'pgSQL/select_implicit.sql'

<details><summary>Diff comparing to 'pgSQL/select_implicit.sql'</summary>
<p>

```diff
diff --git a/home/root1/src/spark/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out b/home/root1/src/spark/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out
index 17303b2..0675820 100755
--- a/home/root1/src/spark/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out
+++ b/home/root1/src/spark/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out
 -91,11 +91,9  struct<>

 -- !query 11
-SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY
-udf(test_missing_target.c)
-ORDER BY udf(c)
+SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c
 -- !query 11 schema
-struct<CAST(udf(cast(c as string)) AS STRING):string,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<c:string,count(1):bigint>
 -- !query 11 output
 ABAB	2
 BBBB	2
 -106,10 +104,9  cccc	2

 -- !query 12
-SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(test_missing_target.c)
-ORDER BY udf(c)
+SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c
 -- !query 12 schema
-struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<count(1):bigint>
 -- !query 12 output
 2
 2
 -120,18 +117,18  struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>

 -- !query 13
-SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b)
+SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b
 -- !query 13 schema
 struct<>
 -- !query 13 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`b`' given input columns: [CAST(udf(cast(count(1) as string)) AS BIGINT)]; line 1 pos 75
+cannot resolve '`b`' given input columns: [count(1)]; line 1 pos 61

 -- !query 14
-SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b)
+SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b
 -- !query 14 schema
-struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<count(1):bigint>
 -- !query 14 output
 1
 2
 -140,10 +137,10  struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>

 -- !query 15
-SELECT udf(test_missing_target.b), udf(count(*))
-  FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b)
+SELECT test_missing_target.b, count(*)
+  FROM test_missing_target GROUP BY b ORDER BY b
 -- !query 15 schema
-struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<b:int,count(1):bigint>
 -- !query 15 output
 1	1
 2	2
 -152,9 +149,9  struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(count(1) as string)

 -- !query 16
-SELECT udf(c) FROM test_missing_target ORDER BY udf(a)
+SELECT c FROM test_missing_target ORDER BY a
 -- !query 16 schema
-struct<CAST(udf(cast(c as string)) AS STRING):string>
+struct<c:string>
 -- !query 16 output
 XXXX
 ABAB
 -169,10 +166,9  CCCC

 -- !query 17
-SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b)
-desc
+SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc
 -- !query 17 schema
-struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<count(1):bigint>
 -- !query 17 output
 4
 3
 -181,17 +177,17  struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>

 -- !query 18
-SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc
+SELECT count(*) FROM test_missing_target ORDER BY 1 desc
 -- !query 18 schema
-struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<count(1):bigint>
 -- !query 18 output
 10

 -- !query 19
-SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1
+SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1
 -- !query 19 schema
-struct<CAST(udf(cast(c as string)) AS STRING):string,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<c:string,count(1):bigint>
 -- !query 19 output
 ABAB	2
 BBBB	2
 -202,30 +198,30  cccc	2

 -- !query 20
-SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3
+SELECT c, count(*) FROM test_missing_target GROUP BY 3
 -- !query 20 schema
 struct<>
 -- !query 20 output
 org.apache.spark.sql.AnalysisException
-GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 63
+GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 53

 -- !query 21
-SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y
-	WHERE udf(x.a) = udf(y.a)
-	GROUP BY udf(b) ORDER BY udf(b)
+SELECT count(*) FROM test_missing_target x, test_missing_target y
+	WHERE x.a = y.a
+	GROUP BY b ORDER BY b
 -- !query 21 schema
 struct<>
 -- !query 21 output
 org.apache.spark.sql.AnalysisException
-Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 14
+Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10

 -- !query 22
-SELECT udf(a), udf(a) FROM test_missing_target
-	ORDER BY udf(a)
+SELECT a, a FROM test_missing_target
+	ORDER BY a
 -- !query 22 schema
-struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(a as string)) AS INT):int>
+struct<a:int,a:int>
 -- !query 22 output
 0	0
 1	1
 -240,10 +236,10  struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(a as string)) AS IN

 -- !query 23
-SELECT udf(udf(a)/2), udf(udf(a)/2) FROM test_missing_target
-	ORDER BY udf(udf(a)/2)
+SELECT a/2, a/2 FROM test_missing_target
+	ORDER BY a/2
 -- !query 23 schema
-struct<CAST(udf(cast((cast(udf(cast(a as string)) as int) div 2) as string)) AS INT):int,CAST(udf(cast((cast(udf(cast(a as string)) as int) div 2) as string)) AS INT):int>
+struct<(a div 2):int,(a div 2):int>
 -- !query 23 output
 0	0
 0	0
 -258,10 +254,10  struct<CAST(udf(cast((cast(udf(cast(a as string)) as int) div 2) as string)) AS

 -- !query 24
-SELECT udf(a/2), udf(a/2) FROM test_missing_target
-	GROUP BY udf(a/2) ORDER BY udf(a/2)
+SELECT a/2, a/2 FROM test_missing_target
+	GROUP BY a/2 ORDER BY a/2
 -- !query 24 schema
-struct<CAST(udf(cast((a div 2) as string)) AS INT):int,CAST(udf(cast((a div 2) as string)) AS INT):int>
+struct<(a div 2):int,(a div 2):int>
 -- !query 24 output
 0	0
 1	1
 -271,11 +267,11  struct<CAST(udf(cast((a div 2) as string)) AS INT):int,CAST(udf(cast((a div 2) a

 -- !query 25
-SELECT udf(x.b), udf(count(*)) FROM test_missing_target x, test_missing_target y
-	WHERE udf(x.a) = udf(y.a)
-	GROUP BY udf(x.b) ORDER BY udf(x.b)
+SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y
+	WHERE x.a = y.a
+	GROUP BY x.b ORDER BY x.b
 -- !query 25 schema
-struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<b:int,count(1):bigint>
 -- !query 25 output
 1	1
 2	2
 -284,11 +280,11  struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(count(1) as string)

 -- !query 26
-SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y
-	WHERE udf(x.a) = udf(y.a)
-	GROUP BY udf(x.b) ORDER BY udf(x.b)
+SELECT count(*) FROM test_missing_target x, test_missing_target y
+	WHERE x.a = y.a
+	GROUP BY x.b ORDER BY x.b
 -- !query 26 schema
-struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
+struct<count(1):bigint>
 -- !query 26 output
 1
 2
 -297,22 +293,22  struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>

 -- !query 27
-SELECT udf(a%2), udf(count(udf(b))) FROM test_missing_target
-GROUP BY udf(test_missing_target.a%2)
-ORDER BY udf(test_missing_target.a%2)
+SELECT a%2, count(b) FROM test_missing_target
+GROUP BY test_missing_target.a%2
+ORDER BY test_missing_target.a%2
 -- !query 27 schema
-struct<CAST(udf(cast((a % 2) as string)) AS INT):int,CAST(udf(cast(count(cast(udf(cast(b as string)) as int)) as string)) AS BIGINT):bigint>
+struct<(a % 2):int,count(b):bigint>
 -- !query 27 output
 0	5
 1	5

 -- !query 28
-SELECT udf(count(c)) FROM test_missing_target
-GROUP BY udf(lower(test_missing_target.c))
-ORDER BY udf(lower(test_missing_target.c))
+SELECT count(c) FROM test_missing_target
+GROUP BY lower(test_missing_target.c)
+ORDER BY lower(test_missing_target.c)
 -- !query 28 schema
-struct<CAST(udf(cast(count(c) as string)) AS BIGINT):bigint>
+struct<count(c):bigint>
 -- !query 28 output
 2
 3
 -321,18 +317,18  struct<CAST(udf(cast(count(c) as string)) AS BIGINT):bigint>

 -- !query 29
-SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b)
+SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b
 -- !query 29 schema
 struct<>
 -- !query 29 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`b`' given input columns: [CAST(udf(cast(count(cast(udf(cast(a as string)) as int)) as string)) AS BIGINT)]; line 1 pos 80
+cannot resolve '`b`' given input columns: [count(a)]; line 1 pos 61

 -- !query 30
-SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2)
+SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2
 -- !query 30 schema
-struct<CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
+struct<count(b):bigint>
 -- !query 30 output
 1
 5
 -340,10 +336,10  struct<CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>

 -- !query 31
-SELECT udf(lower(test_missing_target.c)), udf(count(udf(c)))
-  FROM test_missing_target GROUP BY udf(lower(c)) ORDER BY udf(lower(c))
+SELECT lower(test_missing_target.c), count(c)
+  FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c)
 -- !query 31 schema
-struct<CAST(udf(cast(lower(c) as string)) AS STRING):string,CAST(udf(cast(count(cast(udf(cast(c as string)) as string)) as string)) AS BIGINT):bigint>
+struct<lower(c):string,count(c):bigint>
 -- !query 31 output
 abab	2
 bbbb	3
 -352,9 +348,9  xxxx	1

 -- !query 32
-SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d)))
+SELECT a FROM test_missing_target ORDER BY upper(d)
 -- !query 32 schema
-struct<CAST(udf(cast(a as string)) AS INT):int>
+struct<a:int>
 -- !query 32 output
 0
 1
 -369,33 +365,32  struct<CAST(udf(cast(a as string)) AS INT):int>

 -- !query 33
-SELECT udf(count(b)) FROM test_missing_target
-	GROUP BY udf((b + 1) / 2) ORDER BY udf((b + 1) / 2) desc
+SELECT count(b) FROM test_missing_target
+	GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc
 -- !query 33 schema
-struct<CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
+struct<count(b):bigint>
 -- !query 33 output
 7
 3

 -- !query 34
-SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y
-	WHERE udf(x.a) = udf(y.a)
-	GROUP BY udf(b/2) ORDER BY udf(b/2)
+SELECT count(x.a) FROM test_missing_target x, test_missing_target y
+	WHERE x.a = y.a
+	GROUP BY b/2 ORDER BY b/2
 -- !query 34 schema
 struct<>
 -- !query 34 output
 org.apache.spark.sql.AnalysisException
-Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 14
+Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10

 -- !query 35
-SELECT udf(x.b/2), udf(count(udf(x.b))) FROM test_missing_target x,
-test_missing_target y
-	WHERE udf(x.a) = udf(y.a)
-	GROUP BY udf(x.b/2) ORDER BY udf(x.b/2)
+SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y
+	WHERE x.a = y.a
+	GROUP BY x.b/2 ORDER BY x.b/2
 -- !query 35 schema
-struct<CAST(udf(cast((b div 2) as string)) AS INT):int,CAST(udf(cast(count(cast(udf(cast(b as string)) as int)) as string)) AS BIGINT):bigint>
+struct<(b div 2):int,count(b):bigint>
 -- !query 35 output
 0	1
 1	5
 -403,14 +398,14  struct<CAST(udf(cast((b div 2) as string)) AS INT):int,CAST(udf(cast(count(cast(

 -- !query 36
-SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y
-	WHERE udf(x.a) = udf(y.a)
-	GROUP BY udf(x.b/2)
+SELECT count(b) FROM test_missing_target x, test_missing_target y
+	WHERE x.a = y.a
+	GROUP BY x.b/2
 -- !query 36 schema
 struct<>
 -- !query 36 output
 org.apache.spark.sql.AnalysisException
-Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 21
+Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 13

 -- !query 37
```

</p>
</details>

## How was this patch tested?
Tested as Guided in SPARK-27921

Closes #25350 from Udbhav30/master.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-06 15:14:32 +09:00
HyukjinKwon da3d4b6a35 [SPARK-28537][SQL][HOTFIX][FOLLOW-UP] Add supportColumnar in DebugExec
## What changes were proposed in this pull request?

This PR add supportColumnar in DebugExec. Seems there was a conflict between https://github.com/apache/spark/pull/25274 and https://github.com/apache/spark/pull/25264

Currently tests are broken in Jenkins:

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108687/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108688/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108693/

```
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree: ColumnarToRow +- InMemoryTableScan [id#356956L]       +- InMemoryRelation [id#356956L], StorageLevel(disk, memory, deserialized, 1 replicas)             +- *(1) Range (0, 5, step=1, splits=2)
Stacktrace
sbt.ForkMain$ForkError: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree:
ColumnarToRow
+- InMemoryTableScan [id#356956L]
      +- InMemoryRelation [id#356956L], StorageLevel(disk, memory, deserialized, 1 replicas)
            +- *(1) Range (0, 5, step=1, splits=2)

	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
	at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:431)
	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:323)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:287)
```

## How was this patch tested?

Manually tested the failed test.

Closes #25365 from HyukjinKwon/SPARK-28537.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-06 15:08:15 +09:00
Stavros Kontopoulos 4a2c662315 [SPARK-27921][PYTHON][SQL][TESTS][FOLLOW-UP] Add UDF cases into group by clause in 'udf-group-analytics.sql'
## What changes were proposed in this pull request?

This PR is a followup of a fix as described in here: #25215 (comment)
<details><summary>Diff comparing to 'group-analytics.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
index 3439a05727..de297ab166 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
 -13,9 +13,9  struct<>

 -- !query 1
-SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE
+SELECT udf(a + b), b, udf(SUM(a - b)) FROM testData GROUP BY udf(a + b), b WITH CUBE
 -- !query 1 schema
-struct<(a + b):int,b:int,sum((a - b)):bigint>
+struct<CAST(udf(cast((a + b) as string)) AS INT):int,b:int,CAST(udf(cast(sum(cast((a - b) as bigint)) as string)) AS BIGINT):bigint>
 -- !query 1 output
 2	1	0
 2	NULL	0
 -33,9 +33,9  NULL	NULL	3

 -- !query 2
-SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE
+SELECT udf(a), udf(b), SUM(b) FROM testData GROUP BY udf(a), b WITH CUBE
 -- !query 2 schema
-struct<a:int,b:int,sum(b):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(b as string)) AS INT):int,sum(b):bigint>
 -- !query 2 output
 1	1	1
 1	2	2
 -52,9 +52,9  NULL	NULL	9

 -- !query 3
-SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP
+SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP
 -- !query 3 schema
-struct<(a + b):int,b:int,sum((a - b)):bigint>
+struct<CAST(udf(cast((a + b) as string)) AS INT):int,b:int,sum((a - b)):bigint>
 -- !query 3 output
 2	1	0
 2	NULL	0
 -70,9 +70,9  NULL	NULL	3

 -- !query 4
-SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP
+SELECT udf(a), b, udf(SUM(b)) FROM testData GROUP BY udf(a), b WITH ROLLUP
 -- !query 4 schema
-struct<a:int,b:int,sum(b):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,b:int,CAST(udf(cast(sum(cast(b as bigint)) as string)) AS BIGINT):bigint>
 -- !query 4 output
 1	1	1
 1	2	2
 -97,7 +97,7  struct<>

 -- !query 6
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year
 -- !query 6 schema
 struct<course:string,year:int,sum(earnings):bigint>
 -- !query 6 output
 -111,7 +111,7  dotNET	2013	48000

 -- !query 7
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year)
 -- !query 7 schema
 struct<course:string,year:int,sum(earnings):bigint>
 -- !query 7 output
 -127,9 +127,9  dotNET	2013	48000

 -- !query 8
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year)
+SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year)
 -- !query 8 schema
-struct<course:string,year:int,sum(earnings):bigint>
+struct<course:string,CAST(udf(cast(year as string)) AS INT):int,sum(earnings):bigint>
 -- !query 8 output
 Java	NULL	50000
 NULL	2012	35000
 -138,26 +138,26  dotNET	NULL	63000

 -- !query 9
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course)
+SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course)
 -- !query 9 schema
-struct<course:string,year:int,sum(earnings):bigint>
+struct<course:string,year:int,CAST(udf(cast(sum(cast(earnings as bigint)) as string)) AS BIGINT):bigint>
 -- !query 9 output
 Java	NULL	50000
 dotNET	NULL	63000

 -- !query 10
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year)
+SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year)
 -- !query 10 schema
-struct<course:string,year:int,sum(earnings):bigint>
+struct<CAST(udf(cast(course as string)) AS STRING):string,year:int,sum(earnings):bigint>
 -- !query 10 output
 NULL	2012	35000
 NULL	2013	78000

 -- !query 11
-SELECT course, SUM(earnings) AS sum FROM courseSales
-GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum
+SELECT course, udf(SUM(earnings)) AS sum FROM courseSales
+GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum)
 -- !query 11 schema
 struct<course:string,sum:bigint>
 -- !query 11 output
 -173,7 +173,7  dotNET	63000

 -- !query 12
 SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales
-GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum
+GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum
 -- !query 12 schema
 struct<course:string,sum:bigint,grouping_id(course, earnings):int>
 -- !query 12 output
 -188,10 +188,10  dotNET	63000	1

 -- !query 13
-SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales
+SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales
 GROUP BY CUBE(course, year)
 -- !query 13 schema
-struct<course:string,year:int,grouping(course):tinyint,grouping(year):tinyint,grouping_id(course, year):int>
+struct<CAST(udf(cast(course as string)) AS STRING):string,CAST(udf(cast(year as string)) AS INT):int,grouping(course):tinyint,grouping(year):tinyint,grouping_id(course, year):int>
 -- !query 13 output
 Java	2012	0	0	0
 Java	2013	0	0	0
 -205,7 +205,7  dotNET	NULL	0	1	1

 -- !query 14
-SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year
+SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf(year)
 -- !query 14 schema
 struct<>
 -- !query 14 output
 -214,7 +214,7  grouping() can only be used with GroupingSets/Cube/Rollup;

 -- !query 15
-SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year
+SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY udf(course), year
 -- !query 15 schema
 struct<>
 -- !query 15 output
 -223,7 +223,7  grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 16
-SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year
+SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year)
 -- !query 16 schema
 struct<course:string,year:int,grouping__id:int>
 -- !query 16 output
 -240,7 +240,7  NULL	NULL	3

 -- !query 17
 SELECT course, year FROM courseSales GROUP BY CUBE(course, year)
-HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, year
+HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year)
 -- !query 17 schema
 struct<course:string,year:int>
 -- !query 17 output
 -250,7 +250,7  dotNET	NULL

 -- !query 18
-SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0
+SELECT course, udf(year) FROM courseSales GROUP BY udf(course), year HAVING GROUPING(course) > 0
 -- !query 18 schema
 struct<>
 -- !query 18 output
 -259,7 +259,7  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 19
-SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0
+SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0
 -- !query 19 schema
 struct<>
 -- !query 19 output
 -268,9 +268,9  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 20
-SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0
+SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0
 -- !query 20 schema
-struct<course:string,year:int>
+struct<CAST(udf(cast(course as string)) AS STRING):string,year:int>
 -- !query 20 output
 Java	NULL
 NULL	2012
 -281,7 +281,7  dotNET	NULL

 -- !query 21
 SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year)
-ORDER BY GROUPING(course), GROUPING(year), course, year
+ORDER BY GROUPING(course), GROUPING(year), course, udf(year)
 -- !query 21 schema
 struct<course:string,year:int,grouping(course):tinyint,grouping(year):tinyint>
 -- !query 21 output
 -298,7 +298,7  NULL	NULL	1	1

 -- !query 22
 SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year)
-ORDER BY GROUPING(course), GROUPING(year), course, year
+ORDER BY GROUPING(course), GROUPING(year), course, udf(year)
 -- !query 22 schema
 struct<course:string,year:int,grouping_id(course, year):int>
 -- !query 22 output
 -314,7 +314,7  NULL	NULL	3

 -- !query 23
-SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course)
+SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course)
 -- !query 23 schema
 struct<>
 -- !query 23 output
 -323,7 +323,7  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 24
-SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course)
+SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course)
 -- !query 24 schema
 struct<>
 -- !query 24 output
 -332,7 +332,7  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 25
-SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year
+SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year
 -- !query 25 schema
 struct<course:string,year:int>
 -- !query 25 output
 -348,7 +348,7  NULL	NULL

 -- !query 26
-SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2)
+SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2)
 -- !query 26 schema
 struct<k1:int,k2:int,sum((a - b)):bigint>
 -- !query 26 output
 -368,7 +368,7  NULL	NULL	3

 -- !query 27
-SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b)
+SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b)
 -- !query 27 schema
 struct<k:int,b:int,sum((a - b)):bigint>
 -- !query 27 output
 -386,9 +386,9  NULL	NULL	3

 -- !query 28
-SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k)
+SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k)
 -- !query 28 schema
-struct<(a + b):int,k:int,sum((a - b)):bigint>
+struct<CAST(udf(cast((a + b) as string)) AS INT):int,k:int,sum((a - b)):bigint>
 -- !query 28 output
 NULL	1	3
 NULL	2	0

```

</p>
</details>

## How was this patch tested?
Tested as instructed in SPARK-27921.

Closes #25362 from skonto/group-analytics-followup.

Authored-by: Stavros Kontopoulos <st.kontopoulos@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-06 15:00:28 +09:00
Jungtaek Lim (HeartSaVioR) 128ea37bda [SPARK-28601][CORE][SQL] Use StandardCharsets.UTF_8 instead of "UTF-8" string representation, and get rid of UnsupportedEncodingException
## What changes were proposed in this pull request?

This patch tries to keep consistency whenever UTF-8 charset is needed, as using `StandardCharsets.UTF_8` instead of using "UTF-8". If the String type is needed, `StandardCharsets.UTF_8.name()` is used.

This change also brings the benefit of getting rid of `UnsupportedEncodingException`, as we're providing `Charset` instead of `String` whenever possible.

This also changes some private Catalyst helper methods to operate on encodings as `Charset` objects rather than strings.

## How was this patch tested?

Existing unit tests.

Closes #25335 from HeartSaVioR/SPARK-28601.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-05 20:45:54 -07:00
Wenchen Fan 03e3006312 [SPARK-28213][SQL][FOLLOWUP] code cleanup and bug fix for columnar execution framework
## What changes were proposed in this pull request?

I did a post-hoc review of https://github.com/apache/spark/pull/25008 , and would like to propose some cleanups/fixes/improvements:

1. Do not track the scanTime metrics in `ColumnarToRowExec`. This metrics is specific to file scan, and doesn't make sense for a general batch-to-row operator.
2. Because of 2, we need to track scanTime when building RDDs in the file scan node.
3. use `RDD#mapPartitionsInternal` instead of `flatMap` in several places, as `mapPartitionsInternal` is created for Spark SQL and we use it in almost all the SQL operators.
4. Add `limitNotReachedCond` in `ColumnarToRowExec`. This was in the `ColumnarBatchScan` before and is critical for performance.
5. Clear the relationship between codegen stage and columnar stage. The whole-stage-codegen framework is completely row-based, so these 2 kinds of stages can NEVER overlap. When they are adjacent, it's either a `RowToColumnarExec` above `WholeStageExec`, or a `ColumnarToRowExec` above the `InputAdapter`.
6. Reuse the `ColumnarBatch` in `RowToColumnarExec`. We don't need to create a new one every time, just need to reset it.
7. Do not skip testing full scan node in `LogicalPlanTagInSparkPlanSuite`
8. Add back the removed tests in `WholeStageCodegenSuite`.

## How was this patch tested?

existing tests

Closes #25264 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-06 10:11:18 +08:00
Wenchen Fan 6fb79af48c [SPARK-28344][SQL] detect ambiguous self-join and fail the query
## What changes were proposed in this pull request?

This is an alternative solution of https://github.com/apache/spark/pull/24442 . It fails the query if ambiguous self join is detected, instead of trying to disambiguate it. The problem is that, it's hard to come up with a reasonable rule to disambiguate, the rule proposed by #24442 is mostly a heuristic.

### background of the self-join problem:
This is a long-standing bug and I've seen many people complaining about it in JIRA/dev list.

A typical example:
```
val df1 = …
val df2 = df1.filter(...)
df1.join(df2, df1("a") > df2("a")) // returns empty result
```
The root cause is, `Dataset.apply` is so powerful that users think it returns a column reference which can point to the column of the Dataset at anywhere. This is not true in many cases. `Dataset.apply` returns an `AttributeReference` . Different Datasets may share the same `AttributeReference`. In the example above, `df2` adds a Filter operator above the logical plan of `df1`, and the Filter operator reserves the output `AttributeReference` of its child. This means, `df1("a")` is exactly the same as `df2("a")`, and `df1("a") > df2("a")` always evaluates to false.

### The rule to detect ambiguous column reference caused by self join:
We can reuse the infra in #24442 :
1. each Dataset has a globally unique id.
2. the `AttributeReference` returned by `Dataset.apply` carries the ID and column position(e.g. 3rd column of the Dataset) via metadata.
3. the logical plan of a `Dataset` carries the ID via `TreeNodeTag`

When self-join happens, the analyzer asks the right side plan of join to re-generate output attributes with new exprIds. Based on it, a simple rule to detect ambiguous self join is:
1. find all column references (i.e. `AttributeReference`s with Dataset ID and col position) in the root node of a query plan.
2. for each column reference, traverse the query plan tree, find a sub-plan that carries Dataset ID and the ID is the same as the one in the column reference.
3. get the corresponding output attribute of the sub-plan by the col position in the column reference.
4. if the corresponding output attribute has a different exprID than the column reference, then it means this sub-plan is on the right side of a self-join and has regenerated its output attributes. This is an ambiguous self join because the column reference points to a table being self-joined.

## How was this patch tested?

existing tests and new test cases

Closes #25107 from cloud-fan/new-self-join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-06 10:06:36 +08:00
Kousuke Saruta 794804ea5e [SPARK-28537][SQL] DebugExec cannot debug broadcast or columnar related queries
DebugExec does not implement doExecuteBroadcast and doExecuteColumnar so we can't debug broadcast or columnar related query.

One example for broadcast is here.
```
val df1 = Seq(1, 2, 3).toDF
val df2 = Seq(1, 2, 3).toDF
val joined = df1.join(df2, df1("value") === df2("value"))
joined.debug()

java.lang.UnsupportedOperationException: Debug does not implement doExecuteBroadcast
...
```

Another for columnar is here.
```
val df = Seq(1, 2, 3).toDF
df.persist
df.debug()

java.lang.IllegalStateException: Internal Error class org.apache.spark.sql.execution.debug.package$DebugExec has column support mismatch:
...
```

## How was this patch tested?

Additional test cases in DebuggingSuite.

Closes #25274 from sarutak/fix-debugexec.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-06 08:26:51 +09:00
John Zhuge cae500a255 [SPARK-28178][SQL][FOLLOWUP] DataSourceV2: DataFrameWriter.insertInfo
## What changes were proposed in this pull request?

- DataFrameWriter.insertInto should match column names by position.
- Clean up test cases.

## How was this patch tested?

New tests:
- insertInto: append by position
- insertInto: overwrite partitioned table in static mode by position
- insertInto: overwrite partitioned table in dynamic mode by position

Closes #25353 from jzhuge/SPARK-28178-bypos.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-05 16:05:23 +08:00
Ryan Blue 0345f1174d [SPARK-27661][SQL] Add SupportsNamespaces API
## What changes were proposed in this pull request?

This adds an interface for catalog plugins that exposes namespace operations:
* `listNamespaces`
* `namespaceExists`
* `loadNamespaceMetadata`
* `createNamespace`
* `alterNamespace`
* `dropNamespace`

## How was this patch tested?

API only. Existing tests for regressions.

Closes #24560 from rdblue/SPARK-27661-add-catalog-namespace-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-04 21:29:40 -07:00
Yuming Wang 21a18c6490 [SPARK-28614][SQL][TEST] Do not remove leading write space in the golden result file
## What changes were proposed in this pull request?

It's hard to know if the query needs to be sorted like [`SQLQueryTestSuite.isSorted`](2ecc39c8d3/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L375-L380)) when building a test framework for Thriftserver. So we  can sort both  the `outputs` and  the `expectedOutputs. However, we removed leading write space in the golden result file. This can lead to inconsistent results.
This PR makes it does not remove leading write space in the golden result file. Trailing write space still needs to be removed.

## How was this patch tested?

N/A

Closes #25351 from wangyum/SPARK-28614.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-04 17:43:31 -07:00
Xiao Li 10d4ffd577 [SPARK-28532][SPARK-28530][SQL][FOLLOWUP] Inline doc for FixedPoint(1) batches "Subquery" and "Join Reorder"
## What changes were proposed in this pull request?
Explained why "Subquery" and "Join Reorder" optimization batches should be `FixedPoint(1)`, which was introduced in SPARK-28532 and SPARK-28530.

## How was this patch tested?

Existing UTs.

Closes #25320 from yeshengm/SPARK-28530-followup.

Lead-authored-by: Xiao Li <gatorsmile@gmail.com>
Co-authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-02 14:23:41 -07:00
Sean Owen b148bd5ccb [SPARK-28519][SQL] Use StrictMath log, pow functions for platform independence
## What changes were proposed in this pull request?

See discussion on the JIRA (and dev). At heart, we find that math.log and math.pow can actually return slightly different results across platforms because of hardware optimizations. For the actual SQL log and pow functions, I propose that we should use StrictMath instead to ensure the answers are already the same. (This should have the benefit of helping tests pass on aarch64.)

Further, the atanh function (which is not part of java.lang.Math) can be implemented in a slightly different and more accurate way.

## How was this patch tested?

Existing tests (which will need to be changed).
Some manual testing locally to understand the numeric issues.

Closes #25279 from srowen/SPARK-28519.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-02 10:55:44 -05:00
Yuming Wang efd92993f4 [SPARK-28510][SQL] Implement Spark's own GetFunctionsOperation
## What changes were proposed in this pull request?

This PR implements Spark's own GetFunctionsOperation which mitigates the differences between Spark SQL and Hive UDFs. But our implementation is different from Hive's implementation:
- Our implementation always returns results. Hive only returns results when [(null == catalogName || "".equals(catalogName)) && (null == schemaName || "".equals(schemaName))](https://github.com/apache/hive/blob/rel/release-3.1.1/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java#L101-L119).
- Our implementation pads the `REMARKS` field with the function usage - Hive returns an empty string.
- Our implementation does not support `FUNCTION_TYPE`, but Hive does.

## How was this patch tested?

unit tests

Closes #25252 from wangyum/SPARK-28510.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-02 08:50:42 -07:00
Liang-Chi Hsieh 77c7e91e02 [SPARK-28445][SQL][PYTHON] Fix error when PythonUDF is used in both group by and aggregate expression
## What changes were proposed in this pull request?

When PythonUDF is used in group by, and it is also in aggregate expression, like

```
SELECT pyUDF(a + 1), COUNT(b) FROM testData GROUP BY pyUDF(a + 1)
```

It causes analysis exception in `CheckAnalysis`, like
```
org.apache.spark.sql.AnalysisException: expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function.
```

First, `CheckAnalysis` can't check semantic equality between PythonUDFs.
Second, even we make it possible, runtime exception will be thrown

```
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF1#8615
...
Cause: java.lang.RuntimeException: Couldn't find pythonUDF1#8615 in [cast(pythonUDF0#8614 as int)#8617,count(b#8599)#8607L]
```

The cause is, `ExtractPythonUDFs` extracts both PythonUDFs in group by and aggregate expression. The PythonUDFs are two different aliases now in the logical aggregate. In runtime, we can't bind the resulting expression in aggregate to its grouping and aggregate attributes.

This patch proposes a rule `ExtractGroupingPythonUDFFromAggregate` to extract PythonUDFs in group by and evaluate them before aggregate. We replace the group by PythonUDF in aggregate expression with aliased result.

The query plan of query `SELECT pyUDF(a + 1), pyUDF(COUNT(b)) FROM testData GROUP BY pyUDF(a + 1)`, like

```
== Optimized Logical Plan ==
Project [CAST(pyUDF(cast((a + 1) as string)) AS INT)#8608, cast(pythonUDF0#8616 as bigint) AS CAST(pyUDF(cast(count(b) as string)) AS BIGINT)#8610L]
+- BatchEvalPython [pyUDF(cast(agg#8613L as string))], [pythonUDF0#8616]
   +- Aggregate [cast(groupingPythonUDF#8614 as int)], [cast(groupingPythonUDF#8614 as int) AS CAST(pyUDF(cast((a + 1) as string)) AS INT)#8608, count(b#8599) AS agg#8613L]
      +- Project [pythonUDF0#8615 AS groupingPythonUDF#8614, b#8599]
         +- BatchEvalPython [pyUDF(cast((a#8598 + 1) as string))], [pythonUDF0#8615]
            +- LocalRelation [a#8598, b#8599]

== Physical Plan ==
*(3) Project [CAST(pyUDF(cast((a + 1) as string)) AS INT)#8608, cast(pythonUDF0#8616 as bigint) AS CAST(pyUDF(cast(count(b) as string)) AS BIGINT)#8610L]
+- BatchEvalPython [pyUDF(cast(agg#8613L as string))], [pythonUDF0#8616]
   +- *(2) HashAggregate(keys=[cast(groupingPythonUDF#8614 as int)#8617], functions=[count(b#8599)], output=[CAST(pyUDF(cast((a + 1) as string)) AS INT)#8608, agg#8613L])
      +- Exchange hashpartitioning(cast(groupingPythonUDF#8614 as int)#8617, 5), true
         +- *(1) HashAggregate(keys=[cast(groupingPythonUDF#8614 as int) AS cast(groupingPythonUDF#8614 as int)#8617], functions=[partial_count(b#8599)], output=[cast(groupingPythonUDF#8614 as int)#8617, count#8619L])
            +- *(1) Project [pythonUDF0#8615 AS groupingPythonUDF#8614, b#8599]
               +- BatchEvalPython [pyUDF(cast((a#8598 + 1) as string))], [pythonUDF0#8615]
                  +- LocalTableScan [a#8598, b#8599]
```

## How was this patch tested?

Added tests.

Closes #25215 from viirya/SPARK-28445.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-02 19:47:29 +09:00
Yuming Wang 4e7a4cd20e [SPARK-28521][SQL] Fix error message for built-in functions
## What changes were proposed in this pull request?

```sql
spark-sql> select cast(1);
19/07/26 00:54:17 ERROR SparkSQLDriver: Failed in [select cast(1)]
java.lang.UnsupportedOperationException: empty.init
	at scala.collection.TraversableLike$class.init(TraversableLike.scala:451)
	at scala.collection.mutable.ArrayOps$ofInt.scala$collection$IndexedSeqOptimized$$super$init(ArrayOps.scala:234)
	at scala.collection.IndexedSeqOptimized$class.init(IndexedSeqOptimized.scala:135)
	at scala.collection.mutable.ArrayOps$ofInt.init(ArrayOps.scala:234)
	at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$7$$anonfun$11.apply(FunctionRegistry.scala:565)
	at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$7$$anonfun$11.apply(FunctionRegistry.scala:558)
	at scala.Option.getOrElse(Option.scala:121)
```

The reason is that we did not handle the case [`validParametersCount.length == 0`](2d74f14d74/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L588)) because the [parameter types](2d74f14d74/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L589)) can be `Expression`, `DataType` and `Option`. This PR makes it  handle the case `validParametersCount.length == 0`.

## How was this patch tested?

unit tests

Closes #25261 from wangyum/SPARK-28521.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-01 18:02:50 -05:00
Wing Yew Poon 80ab19b9fd [SPARK-26329][CORE] Faster polling of executor memory metrics.
## What changes were proposed in this pull request?

Prior to this change, in an executor, on each heartbeat, memory metrics are polled and sent in the heartbeat. The heartbeat interval is 10s by default. With this change, in an executor, memory metrics can optionally be polled in a separate poller at a shorter interval.

For each executor, we use a map of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks) to track what stages are active as well as the per-stage memory metric peaks. When polling the executor memory metrics, we attribute the memory to the active stage(s), and update the peaks. In a heartbeat, we send the per-stage peaks (for stages active at that time), and then reset the peaks. The semantics would be that the per-stage peaks sent in each heartbeat are the peaks since the last heartbeat.

We also keep a map of taskId to memory metric peaks. This tracks the metric peaks during the lifetime of the task. The polling thread updates this as well. At end of a task, we send the peak metric values in the task result. In case of task failure, we send the peak metric values in the `TaskFailedReason`.

We continue to do the stage-level aggregation in the EventLoggingListener.

For the driver, we still only poll on heartbeats. What the driver sends will be the current values of the metrics in the driver at the time of the heartbeat. This is semantically the same as before.

## How was this patch tested?

Unit tests. Manually tested applications on an actual system and checked the event logs; the metrics appear in the SparkListenerTaskEnd and SparkListenerStageExecutorMetrics events.

Closes #23767 from wypoon/wypoon_SPARK-26329.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-08-01 09:09:46 -05:00
WeichenXu 26d03b62e2 [SPARK-28366][CORE] Logging in driver when loading single large unsplittable file
## What changes were proposed in this pull request?

Logging in driver when loading single large unsplittable file via `sc.textFile` or csv/json datasouce.
Current condition triggering logging is
* only generate one partition
* file is unsplittable, possible reason is:
   - compressed by unsplittable compression algo such as gzip.
   - multiLine mode in csv/json datasource
   - wholeText mode in text datasource
* file size exceed the config threshold `spark.io.warning.largeFileThreshold` (default value is 1GB)

## How was this patch tested?

Manually test.
Generate one gzip file exceeding 1GB,
```
base64 -b 50 /dev/urandom | head -c 2000000000 > file1.txt
cat file1.txt | gzip > file1.gz
```
then launch spark-shell,

run
```
sc.textFile("file:///path/to/file1.gz").count()
```
Will print log like:
```
WARN HadoopRDD: Loading one large unsplittable file file:/.../f1.gz with only one partition, because the file is compressed by unsplittable compression codec
```

run
```
sc.textFile("file:///path/to/file1.txt").count()
```
Will print log like:
```
WARN HadoopRDD: Loading one large file file:/.../f1.gz with only one partition, we can increase partition numbers by the `minPartitions` argument in method `sc.textFile
```

run
```
spark.read.csv("file:///path/to/file1.gz").count
```
Will print log like:
```
WARN CSVScan: Loading one large unsplittable file file:/.../f1.gz with only one partition, the reason is: the file is compressed by unsplittable compression codec
```

run
```
spark.read.option("multiLine", true).csv("file:///path/to/file1.gz").count
```
Will print log like:
```
WARN CSVScan: Loading one large unsplittable file file:/.../f1.gz with only one partition, the reason is: the csv datasource is set multiLine mode
```

JSON and Text datasource also tested with similar cases.

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25134 from WeichenXu123/log_gz.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-01 20:29:18 +08:00
Marco Gaido ee41001949 [SPARK-26218][SQL] Overflow on arithmetic operations returns incorrect result
## What changes were proposed in this pull request?

When an overflow occurs performing an arithmetic operation, we are returning an incorrect value. Instead, we should throw an exception, as stated in the SQL standard.

## How was this patch tested?

added UT + existing UTs (improved)

Closes #21599 from mgaido91/SPARK-24598.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-01 14:51:38 +08:00
Yuming Wang 3002a3bf3c [SPARK-28581][SQL] Replace _FUNC_ in UDF ExpressionInfo
## What changes were proposed in this pull request?

This PR moves `replaceFunctionName(usage: String, functionName: String)`
from `DescribeFunctionCommand` to `ExpressionInfo` in order to make `ExpressionInfo` returns actual name instead of placeholder. We can get `ExpressionInfo`s directly through `SessionCatalog.lookupFunctionInfo` API and get the real names.

## How was this patch tested?

unit tests

Closes #25314 from wangyum/SPARK-28581.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-31 13:08:49 -07:00
gengjiaan d03ec65f01 [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax
## What changes were proposed in this pull request?

This PR aims to support ANSI SQL `Boolean-Predicate` syntax.
```sql
expression IS [NOT] TRUE
expression IS [NOT] FALSE
expression IS [NOT] UNKNOWN
```

There are some mainstream database support this syntax.
- **PostgreSQL:**  https://www.postgresql.org/docs/9.1/functions-comparison.html
- **Hive:** https://issues.apache.org/jira/browse/HIVE-13583
- **Redshift:** https://docs.aws.amazon.com/redshift/latest/dg/r_Boolean_type.html
- **Vertica:** https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Predicates/Boolean-predicate.htm

For example:
```sql
spark-sql> select null is true, null is not true;
false	true

spark-sql> select false is true, false is not true;
false	true

spark-sql> select true is true, true is not true;
true	false

spark-sql> select null is false, null is not false;
false	true

spark-sql> select false is false, false is not false;
true	false

spark-sql> select true is false,  true is not false;
false	true

spark-sql> select null is unknown, null is not unknown;
true	false

spark-sql> select false is unknown, false is not unknown;
false	true

spark-sql> select true is unknown, true is not unknown;
false	true
```
**Note**: A null input is treated as the logical value "unknown".

## How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #25074 from beliefer/ansi-sql-boolean-test.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-30 23:59:50 -07:00
Yuming Wang 261e113449 [SPARK-28038][SQL][TEST] Port text.sql
## What changes were proposed in this pull request?

This PR is to port text.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/text.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/text.out

When porting the test cases, found a PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28037](https://issues.apache.org/jira/browse/SPARK-28037): Add built-in String Functions: quote_literal

Also, found three inconsistent behavior:
[SPARK-27930](https://issues.apache.org/jira/browse/SPARK-27930): Spark SQL's format_string can not fully support PostgreSQL's format
[SPARK-28036](https://issues.apache.org/jira/browse/SPARK-28036):  Built-in udf left/right has inconsistent behavior
[SPARK-28033](https://issues.apache.org/jira/browse/SPARK-28033): String concatenation should low priority than other operators

## How was this patch tested?

N/A

Closes #24862 from wangyum/SPARK-28038.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-31 11:36:26 +09:00
Dilip Biswal ee3c1c777d [SPARK-28375][SQL] Make pullupCorrelatedPredicate idempotent
## What changes were proposed in this pull request?

This PR makes the optimizer rule PullupCorrelatedPredicates idempotent.
## How was this patch tested?

A new test PullupCorrelatedPredicatesSuite

Closes #25268 from dilipbiswal/pr-25164.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-30 16:29:24 -07:00
Yuming Wang 2656c9d304 [SPARK-28071][SQL][TEST] Port strings.sql
## What changes were proposed in this pull request?

This PR is to port strings.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/strings.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/strings.out

When porting the test cases, found nine PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28076](https://issues.apache.org/jira/browse/SPARK-28076): Support regular expression substring
[SPARK-28078](https://issues.apache.org/jira/browse/SPARK-28078):  Add support other 4 REGEXP functions
[SPARK-28412](https://issues.apache.org/jira/browse/SPARK-28412): OVERLAY function support byte array
[SPARK-28083](https://issues.apache.org/jira/browse/SPARK-28083):  ANSI SQL: LIKE predicate: ESCAPE clause
[SPARK-28087](https://issues.apache.org/jira/browse/SPARK-28087):  Add support split_part
[SPARK-28122](https://issues.apache.org/jira/browse/SPARK-28122): Missing `sha224`/`sha256 `/`sha384 `/`sha512 ` functions
[SPARK-28123](https://issues.apache.org/jira/browse/SPARK-28123): Add support string functions: btrim
[SPARK-28448](https://issues.apache.org/jira/browse/SPARK-28448): Implement ILIKE operator
[SPARK-28449](https://issues.apache.org/jira/browse/SPARK-28449): Missing escape_string_warning and standard_conforming_strings config

Also, found five inconsistent behavior:
[SPARK-27952](https://issues.apache.org/jira/browse/SPARK-27952): String Functions: regexp_replace is not compatible
[SPARK-28121](https://issues.apache.org/jira/browse/SPARK-28121): decode can not accept 'escape' as charset
[SPARK-27930](https://issues.apache.org/jira/browse/SPARK-27930): Replace `strpos` with `locate` or `position` in Spark SQL
[SPARK-27930](https://issues.apache.org/jira/browse/SPARK-27930): Replace `to_hex` with `hex ` or in Spark SQL
[SPARK-28451](https://issues.apache.org/jira/browse/SPARK-28451): `substr` returns different values

## How was this patch tested?

N/A

Closes #24923 from wangyum/SPARK-28071.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-07-30 18:54:14 +09:00
John Zhuge 749b1d3a45 [SPARK-28178][SQL] DataSourceV2: DataFrameWriter.insertInfo
## What changes were proposed in this pull request?

Support multiple catalogs in the following InsertInto use cases:

- DataFrameWriter.insertInto("catalog.db.tbl")

Support matrix:

SaveMode|Partitioned Table|Partition Overwrite Mode|Action
--------|-----------------|------------------------|------
Append|*|*|AppendData
Overwrite|no|*|OverwriteByExpression(true)
Overwrite|yes|STATIC|OverwriteByExpression(true)
Overwrite|yes|DYNAMIC|OverwritePartitionsDynamic

## How was this patch tested?

New tests.
All existing catalyst and sql/core tests.

Closes #24980 from jzhuge/SPARK-28178-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-30 17:22:33 +08:00
Yuming Wang df84bfe6fb [SPARK-28406][SQL][TEST] Port union.sql
## What changes were proposed in this pull request?

This PR is to port union.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/union.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/union.out

When porting the test cases, found four PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28409](https://issues.apache.org/jira/browse/SPARK-28409): SELECT FROM syntax
[SPARK-28298](https://issues.apache.org/jira/browse/SPARK-28298): Fully support char and varchar types
[SPARK-28557](https://issues.apache.org/jira/browse/SPARK-28557): Support empty select list
[SPARK-27767](https://issues.apache.org/jira/browse/SPARK-27767): Built-in function: generate_series

## How was this patch tested?

N/A

Closes #25163 from wangyum/SPARK-28406.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-30 00:14:17 -07:00
Yuming Wang d530d86ab8 [SPARK-28326][SQL][TEST] Port join.sql
## What changes were proposed in this pull request?

This PR is to port join.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/join.out

When porting the test cases, found nine PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-27877](https://issues.apache.org/jira/browse/SPARK-27877): ANSI SQL: LATERAL derived table(T491)
[SPARK-20856](https://issues.apache.org/jira/browse/SPARK-20856): support statement using nested joins
[SPARK-27987](https://issues.apache.org/jira/browse/SPARK-27987): Support POSIX Regular Expressions
[SPARK-28382](https://issues.apache.org/jira/browse/SPARK-28382): Array Functions: unnest
[SPARK-25411](https://issues.apache.org/jira/browse/SPARK-25411): Implement range partition in Spark
[SPARK-28377](https://issues.apache.org/jira/browse/SPARK-28377): Fully support correlation names in the FROM clause
[SPARK-28330](https://issues.apache.org/jira/browse/SPARK-28330): Enhance query limit
[SPARK-28379](https://issues.apache.org/jira/browse/SPARK-28379): Correlated scalar subqueries must be aggregated
[SPARK-16452](https://issues.apache.org/jira/browse/SPARK-16452): basic INFORMATION_SCHEMA support

## How was this patch tested?

N/A

Closes #25148 from wangyum/SPARK-28326.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-30 00:09:56 -07:00
Shixiong Zhu 196a4d7117 [SPARK-28556][SQL] QueryExecutionListener should also notify Error
## What changes were proposed in this pull request?

Right now `Error` is not sent to `QueryExecutionListener.onFailure`. If there is any `Error` (such as `AssertionError`) when running a query, `QueryExecutionListener.onFailure` cannot be triggered.

This PR changes `onFailure` to accept a `Throwable` instead.

## How was this patch tested?

Jenkins

Closes #25292 from zsxwing/fix-QueryExecutionListener.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-30 11:47:36 +09:00
Maxim Gekk caa23e3efd [SPARK-28459][SQL] Add make_timestamp function
## What changes were proposed in this pull request?

New function `make_timestamp()` takes 6 columns `year`, `month`, `day`, `hour`, `min`, `sec` + optionally `timezone`, and makes new column of the `TIMESTAMP` type. If values in the input columns are `null` or out of valid ranges, the function returns `null`. Valid ranges are:
- `year` - `[1, 9999]`
- `month` - `[1, 12]`
- `day` - `[1, 31]`
- `hour` - `[0, 23]`
- `min` - `[0, 59]`
- `sec` - `[0, 60]`. If the `sec` argument equals to 60, the seconds field is set to 0 and 1 minute is added to the final timestamp.
- `timezone` - an identifier of timezone. Actual database of timezones can be found there: https://www.iana.org/time-zones.

Also constructed timestamp must be valid otherwise `make_timestamp` returns `null`.

The function is implemented similarly to `make_timestamp` in PostgreSQL: https://www.postgresql.org/docs/11/functions-datetime.html to maintain feature parity with it.

Here is an example:
```sql
select make_timestamp(2014, 12, 28, 6, 30, 45.887);
  2014-12-28 06:30:45.887
select make_timestamp(2014, 12, 28, 6, 30, 45.887, 'CET');
  2014-12-28 10:30:45.887
select make_timestamp(2019, 6, 30, 23, 59, 60)
  2019-07-01 00:00:00
```

Returned value has Spark Catalyst type `TIMESTAMP` which is similar to Oracle's `TIMESTAMP WITH LOCAL TIME ZONE` (see https://docs.oracle.com/cd/B28359_01/server.111/b28298/ch4datetime.htm#i1006169) where data is stored in the session time zone, and the time zone offset is not stored as part of the column data. When users retrieve the data, Spark returns it in the session time zone specified by the SQL config `spark.sql.session.timeZone`.

## How was this patch tested?

Added new tests to `DateExpressionsSuite`, and uncommented a test for `make_timestamp` in `pgSQL/timestamp.sql`.

Closes #25220 from MaxGekk/make_timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-29 11:00:08 -07:00
Lee Dongjin d98aa2a184 [MINOR] Trivial cleanups
These are what I found during working on #22282.

- Remove unused value: `UnsafeArraySuite#defaultTz`
- Remove redundant new modifier to the case class, `KafkaSourceRDDPartition`
- Remove unused variables from `RDD.scala`
- Remove trailing space from `structured-streaming-kafka-integration.md`
- Remove redundant parameter from `ArrowConvertersSuite`: `nullable` is `true` by default.
- Remove leading empty line: `UnsafeRow`
- Remove trailing empty line: `KafkaTestUtils`
- Remove unthrown exception type: `UnsafeMapData`
- Replace unused declarations: `expressions`
- Remove duplicated default parameter: `AnalysisErrorSuite`
- `ObjectExpressionsSuite`: remove duplicated parameters, conversions and unused variable

Closes #25251 from dongjinleekr/cleanup/201907.

Authored-by: Lee Dongjin <dongjin@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 23:38:02 +09:00
Dongjoon Hyun 18156d5503 [SPARK-28086][SQL] Add a function alias random for Rand
## What changes were proposed in this pull request?

This PR aims to add a SQL function alias `random` to the existing `rand` function.
Please note that this adds the alias to SQL layer only because this is for PostgreSQL feature parity.

- [PostgreSQL Random function](https://www.postgresql.org/docs/11/functions-math.html)
- [SPARK-23160 Port window.sql](https://github.com/apache/spark/pull/24881/files#diff-14489bae6b27814d4cde0456a7ae75c8R702)
- [SPARK-28406 Port union.sql](https://github.com/apache/spark/pull/25163/files#diff-23a3430e0e1ff88830cbb43701da1f2cR402)

## How was this patch tested?

Manual.
```sql
spark-sql> DESCRIBE FUNCTION random;
Function: random
Class: org.apache.spark.sql.catalyst.expressions.Rand
Usage: random([seed]) - Returns a random value with independent and identically distributed (i.i.d.) uniformly distributed values in [0, 1).
```

Closes #25282 from dongjoon-hyun/SPARK-28086.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 20:17:30 +09:00
Maxim Gekk a5a5da78cf [SPARK-28471][SQL] Replace yyyy by uuuu in date-timestamp patterns without era
## What changes were proposed in this pull request?

In the PR, I propose to use `uuuu` for years instead of `yyyy` in date/timestamp patterns without the era pattern `G` (https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html). **Parsing/formatting of positive years (current era) will be the same.** The difference is in formatting negative years belong to previous era - BC (Before Christ).

I replaced the `yyyy` pattern by `uuuu` everywhere except:
1. Test, Suite & Benchmark. Existing tests must work as is.
2. `SimpleDateFormat` because it doesn't support the `uuuu` pattern.
3. Comments and examples (except comments related to already replaced patterns).

Before the changes, the year of common era `100` and the year of BC era `-99`, showed similarly as `100`.  After the changes negative years will be formatted with the `-` sign.

Before:
```Scala
scala> Seq(java.time.LocalDate.of(-99, 1, 1)).toDF().show
+----------+
|     value|
+----------+
|0100-01-01|
+----------+
```

After:
```Scala
scala> Seq(java.time.LocalDate.of(-99, 1, 1)).toDF().show
+-----------+
|      value|
+-----------+
|-0099-01-01|
+-----------+
```

## How was this patch tested?

By existing test suites, and added tests for negative years to `DateFormatterSuite` and `TimestampFormatterSuite`.

Closes #25230 from MaxGekk/year-pattern-uuuu.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-28 20:36:36 -07:00
Dongjoon Hyun a428f40669 [SPARK-28549][BUILD][CORE][SQL] Use text.StringEscapeUtils instead lang3.StringEscapeUtils
## What changes were proposed in this pull request?

`org.apache.commons.lang3.StringEscapeUtils` was deprecated over two years ago at [LANG-1316](https://issues.apache.org/jira/browse/LANG-1316). There is no bug fixes after that.
```java
/**
 * <p>Escapes and unescapes {code String}s for
 * Java, Java Script, HTML and XML.</p>
 *
 * <p>#ThreadSafe#</p>
 * since 2.0
 * deprecated as of 3.6, use commons-text
 * <a href="https://commons.apache.org/proper/commons-text/javadocs/api-release/org/apache/commons/text/StringEscapeUtils.html">
 * StringEscapeUtils</a> instead
 */
Deprecated
public class StringEscapeUtils {
```

This PR aims to use the latest one from `commons-text` module which has more bug fixes like
[TEXT-100](https://issues.apache.org/jira/browse/TEXT-100), [TEXT-118](https://issues.apache.org/jira/browse/TEXT-118) and [TEXT-120](https://issues.apache.org/jira/browse/TEXT-120) by the following replacement.
```scala
-import org.apache.commons.lang3.StringEscapeUtils
+import org.apache.commons.text.StringEscapeUtils
```

This will add a new dependency to `hadoop-2.7` profile distribution. In `hadoop-3.2` profile, we already have it.
```
+commons-text-1.6.jar
```

## How was this patch tested?

Pass the Jenkins with the existing tests.
- [Hadoop 2.7](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108281)
- [Hadoop 3.2](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108282)

Closes #25281 from dongjoon-hyun/SPARK-28549.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 11:45:29 +09:00
Kousuke Saruta 6bc5c6a4e7 [SPARK-28520][SQL] WholeStageCodegen does not work property for LocalTableScanExec
Code is not generated for LocalTableScanExec although proper situations.

If a LocalTableScanExec plan has the direct parent plan which supports WholeStageCodegen,
the LocalTableScanExec plan also should be within a WholeStageCodegen domain.
But code is not generated for LocalTableScanExec and InputAdapter is inserted for now.

```
val df1 = spark.createDataset(1 to 10).toDF
val df2 = spark.createDataset(1 to 10).toDF
val df3 = df1.join(df2, df1("value") === df2("value"))
df3.explain(true)

...

== Physical Plan ==
*(1) BroadcastHashJoin [value#1], [value#6], Inner, BuildRight
:- LocalTableScan [value#1]                                             // LocalTableScanExec is not within a WholeStageCodegen domain
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   +- LocalTableScan [value#6]
```

```
scala> df3.queryExecution.executedPlan.children.head.children.head.getClass
res4: Class[_ <: org.apache.spark.sql.execution.SparkPlan] = class org.apache.spark.sql.execution.InputAdapter
```

For the current implementation of LocalTableScanExec, codegen is enabled in case `parent` is not null
but `parent` is set in `consume`, which is called after `insertInputAdapter` so it doesn't work as intended.

After applying this cnahge, we can get following plan, which means LocalTableScanExec is within a WholeStageCodegen domain.

```
== Physical Plan ==
*(1) BroadcastHashJoin [value#63], [value#68], Inner, BuildRight
:- *(1) LocalTableScan [value#63]
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   +- LocalTableScan [value#68]

## How was this patch tested?

New test cases are added into WholeStageCodegenSuite.

Closes #25260 from sarutak/localtablescan-improvement.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-07-29 08:35:25 +09:00
Huaxin Gao 3c5278748d [SPARK-28277][SQL][PYTHON][TESTS][FOLLOW-UP] Re-enable commented out test
## What changes were proposed in this pull request?

Fix for ```SPARK-28441 (PythonUDF used in correlated scalar subquery causes UnsupportedOperationException)``` is in. Re-enable the commented out test for ```udf(max(udf(column))) ```

## How was this patch tested?

use existing test ```udf-except.sql```

Closes #25278 from huaxingao/spark-28277n.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-28 15:52:31 -07:00
shahid 485ae6d181 [SPARK-25474][SQL] Support spark.sql.statistics.fallBackToHdfs in data source tables
In case of CatalogFileIndex datasource table, sizeInBytes is always coming as default size in bytes, which is  8.0EB (Even when the user give fallBackToHdfsForStatsEnabled=true) . So, the datasource table which has CatalogFileIndex, always prefer SortMergeJoin, instead of BroadcastJoin, even though the size is below broadcast join threshold.
In this PR, In case of CatalogFileIndex table, if we enable "fallBackToHdfsForStatsEnabled=true", then the computeStatistics  get the sizeInBytes from the hdfs and we get the actual size of the table. Hence, during join operation, when the table size is below broadcast threshold, it will prefer broadCastHashJoin instead of SortMergeJoin.

Added UT

Closes #22502 from shahidki31/SPARK-25474.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-28 15:35:37 -07:00
Dongjoon Hyun d943ee0a88 [SPARK-28545][SQL] Add the hash map size to the directional log of ObjectAggregationIterator
## What changes were proposed in this pull request?

`ObjectAggregationIterator` shows a directional info message to increase `spark.sql.objectHashAggregate.sortBased.fallbackThreshold` when the size of the in-memory hash map grows too large and it falls back to sort-based aggregation.
However, we don't know how much we need to increase. This PR adds the size of the current in-memory hash map size to the log message.

**BEFORE**
```
15:21:41.669 Executor task launch worker for task 0 INFO
ObjectAggregationIterator: Aggregation hash map reaches threshold capacity (2 entries), ...
```

**AFTER**
```
15:20:05.742 Executor task launch worker for task 0 INFO
ObjectAggregationIterator: Aggregation hash map size 2 reaches threshold capacity (2 entries), ...
```

## How was this patch tested?

Manual. For example, run `ObjectHashAggregateSuite.scala`'s `typed_count fallback to sort-based aggregation` and search the above message in `target/unit-tests.log`.

Closes #25276 from dongjoon-hyun/SPARK-28545.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-27 18:55:36 -07:00
Yuming Wang 8255bd2937 [SPARK-28460][SQL][TEST][test-hadoop3.2] Port test from HIVE-11835
## What changes were proposed in this pull request?

[HIVE-11835](https://issues.apache.org/jira/browse/HIVE-11835) fixed type `decimal(1,1)` reads 0.0, 0.00, etc from text file as NULL. We fixed this issue after upgrade the build-in Hive to 2.3.5. This PR port the test from [HIVE-11835](https://issues.apache.org/jira/browse/HIVE-11835).

Hive test result:
https://github.com/apache/hive/blob/release-2.3.5-rc0/ql/src/test/results/clientpositive/decimal_1_1.q.out#L67-L96

## How was this patch tested?

N/A

Closes #25212 from wangyum/SPARK-28460.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-27 17:04:27 -07:00
Yuming Wang 9eb541be22 [SPARK-28424][SQL] Support typed interval expression
## What changes were proposed in this pull request?

This PR add support typed `interval` expression:
```sql
spark-sql> select interval 'interval 3 year 1 hour';
interval 3 years 1 hours
spark-sql>
```

Please note that this pr did not add a cast alias for `interval` type like [other types](2d74f14d74/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L529-L541)) because neither PostgreSQL nor Hive supports this syntax.

## How was this patch tested?

unit tests

Closes #25241 from wangyum/SPARK-28424.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-27 14:25:35 -07:00
HyukjinKwon 8ce1ae52db [SPARK-28536][SQL][PYTHON][TESTS] Reduce shuffle partitions in Python UDF tests in SQLQueryTestSuite
## What changes were proposed in this pull request?

In Python UDF tests, the number of shuffle partitions matters considerably in the testing time because it requires to fork and communicate between external processes.

**Before:**

![image](https://user-images.githubusercontent.com/6477701/61989374-465c0080-b069-11e9-9936-b386d0cccf7a.png)

**After: (with 4)**

![Screen Shot 2019-07-27 at 10 43 34 AM](https://user-images.githubusercontent.com/9700541/61997757-743a4880-b05b-11e9-9180-8d0976bda3bd.png)

## How was this patch tested?

Manually tested in my local.

**Before:**

```
[info] SQLQueryTestSuite:
[info] - udf/udf-window.sql - Scala UDF (58 seconds, 558 milliseconds)
[info] - udf/udf-window.sql - Regular Python UDF (58 seconds, 371 milliseconds)
[info] - udf/udf-window.sql - Scalar Pandas UDF (1 minute, 8 seconds)
```

**After:**

```
[info] SQLQueryTestSuite:
[info] - udf/udf-window.sql - Scala UDF (14 seconds, 690 milliseconds)
[info] - udf/udf-window.sql - Regular Python UDF (10 seconds, 467 milliseconds)
[info] - udf/udf-window.sql - Scalar Pandas UDF (10 seconds, 895 milliseconds)
```

Closes #25271 from HyukjinKwon/SPARK-28536.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-27 10:46:35 -07:00
HyukjinKwon 1856ee3b92 [SPARK-28441][SQL][TESTS][FOLLOW-UP] Skip Python tests if python executable and pyspark library are unavailable
##  What changes were proposed in this pull request?

We should add `assume(shouldTestPythonUDFs)`. Maybe it's not a biggie in general but it can matter in other venders' testing base. For instance, if somebody launches a test in a minimal docker image, it might make the tests failed suddenly.

This skipping stuff isn't completely new in our test base. See `TestUtils.testCommandAvailable` for instance.

## How was this patch tested?

Manually tested.

Closes #25272 from HyukjinKwon/SPARK-28441.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-27 15:56:12 +09:00
Yesheng Ma d4e246658a [SPARK-28530][SQL] Cost-based join reorder optimizer batch should be FixedPoint(1)
## What changes were proposed in this pull request?
Since for AQP the cost for joins can change between multiple runs, there is no reason that we have an idempotence enforcement on this optimizer batch. We thus make it `FixedPoint(1)` instead of `Once`.

## How was this patch tested?
Existing UTs.

Closes #25266 from yeshengm/SPARK-28530.

Lead-authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 22:57:39 -07:00
Yesheng Ma e037a11494 [SPARK-28532][SQL] Make optimizer batch "subquery" FixedPoint(1)
## What changes were proposed in this pull request?
In the Catalyst optimizer, the batch subquery actually calls the optimizer recursively. Therefore it makes no sense to enforce idempotence on it and we change this batch to `FixedPoint(1)`.

## How was this patch tested?
Existing UTs.

Closes #25267 from yeshengm/SPARK-28532.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 22:48:42 -07:00
Liang-Chi Hsieh 558dd23601 [SPARK-28441][SQL][PYTHON] Fix error when non-foldable expression is used in correlated scalar subquery
## What changes were proposed in this pull request?

In SPARK-15370, We checked the expression at the root of the correlated subquery, in order to fix count bug. If a `PythonUDF` in in the checking path, evaluating it causes the failure as we can't statically evaluate `PythonUDF`. The Python UDF test added at SPARK-28277 shows this issue.

If we can statically evaluate the expression, we intercept NULL values coming from the outer join and replace them with the value that the subquery's expression like before, if it is not, we replace them with the `PythonUDF` expression, with statically evaluated parameters.

After this, the last query in `udf-except.sql` which throws `java.lang.UnsupportedOperationException` can be run:

```
SELECT t1.k
FROM   t1
WHERE  t1.v <= (SELECT   udf(max(udf(t2.v)))
                FROM     t2
                WHERE    udf(t2.k) = udf(t1.k))
MINUS
SELECT t1.k
FROM   t1
WHERE  udf(t1.v) >= (SELECT   min(udf(t2.v))
                FROM     t2
                WHERE    t2.k = t1.k)
-- !query 2 schema
struct<k:string>
-- !query 2 output
two
```

Note that this issue is also for other non-foldable expressions, like rand. As like PythonUDF, we can't call `eval` on this kind of expressions in optimization. The evaluation needs to defer to query runtime.

## How was this patch tested?

Added tests.

Closes #25204 from viirya/SPARK-28441.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-27 10:38:34 +08:00
Yuming Wang 836a8ff2b9 [SPARK-28518][SQL][TEST] Refer to ChecksumFileSystem#isChecksumFile to fix StatisticsCollectionTestBase#getDataSize
## What changes were proposed in this pull request?

This PR fix [StatisticsCollectionTestBase.getDataSize](8158d5e27f/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala (L298-L304)) refer to [ChecksumFileSystem.isChecksumFile](https://github.com/apache/hadoop/blob/release-2.7.4-RC0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java#L93-L97).

More details: https://github.com/apache/spark/pull/25014#discussion_r307050435

## How was this patch tested?

unit tests

Closes #25259 from wangyum/SPARK-28518.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-26 14:48:24 -07:00
Yuming Wang 545c7ee00b [SPARK-28463][SQL] Thriftserver throws BigDecimal incompatible with HiveDecimal
## What changes were proposed in this pull request?

How to reproduce this issue:
```shell
build/sbt clean package -Phive -Phive-thriftserver -Phadoop-3.2
export SPARK_PREPEND_CLASSES=true
sbin/start-thriftserver.sh

[rootspark-3267648 spark]# bin/beeline -u jdbc:hive2://localhost:10000/default -e "select cast(1 as decimal(38, 18));"
Connecting to jdbc:hive2://localhost:10000/default
Connected to: Spark SQL (version 3.0.0-SNAPSHOT)
Driver: Hive JDBC (version 2.3.5)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Error: java.lang.ClassCastException: java.math.BigDecimal incompatible with org.apache.hadoop.hive.common.type.HiveDecimal (state=,code=0)
Closing: 0: jdbc:hive2://localhost:10000/default
```

This pr fix this issue.

## How was this patch tested?

unit tests

Closes #25217 from wangyum/SPARK-28463.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 10:30:01 -07:00
Yuming Wang 6807a82047 [SPARK-28524][SQL] Fix ThriftServerTab lost error message
## What changes were proposed in this pull request?

The ThriftServerTab lost the error message since [SPARK-28260](https://issues.apache.org/jira/browse/SPARK-28260):
![image](https://user-images.githubusercontent.com/5399861/61964309-27755400-b000-11e9-8bc4-b5bb01d2b0e6.png)
![image](https://user-images.githubusercontent.com/5399861/61964588-cf8b1d00-b000-11e9-9583-2f14bdb114a2.png)

This pr fix this issue.

## How was this patch tested?

manual tests
![image](https://user-images.githubusercontent.com/5399861/61965964-11699280-b004-11e9-83e8-688e3ef8727f.png)
![image](https://user-images.githubusercontent.com/5399861/61965940-09115780-b004-11e9-9f1c-fe9bfcb38128.png)

Closes #25263 from wangyum/SPARK-28524.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 09:49:58 -07:00
Yesheng Ma c93d2dd183 [SPARK-28237][SQL] Enforce Idempotence for Once batches in RuleExecutor
## What changes were proposed in this pull request?
In adaptive query processing (AQE), query plans are optimized on the fly during execution. However, a few `Once` rules can be problematic for such optimization since they can either generate wrong plan/unnecessary intermediate plan nodes.

This PR enforces idempotence for "Once" batches that are supposed to run once. This is a key enabler for AQE re-optimization and can improve robustness for existing optimizer rules.

Once batches that are currently not idempotent are marked in a blacklist. We will submit followup PRs to fix idempotence of these rules.

## How was this patch tested?
Existing UTs. Failing Once rules are temporarily blacklisted.

Closes #25249 from yeshengm/idempotence-checker.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-25 23:44:56 -07:00
Yiheng Wang 6361467bde [SPARK-28289][SQL][PYTHON][TESTS] Convert and port 'union.sql' into UDF test base
## What changes were proposed in this pull request?
This PR adds some tests converted from 'union.sql' to test UDFs

<details><summary>Diff comparing to 'union.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out
index b023df825d..84b5e10dbe 100644
--- a/sql/core/src/test/resources/sql-tests/results/union.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out
 -19,10 +19,10  struct<>

 -- !query 2
-SELECT *
-FROM   (SELECT * FROM t1
+SELECT udf(c1) as c1, udf(c2) as c2
+FROM   (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1
         UNION ALL
-        SELECT * FROM t1)
+        SELECT udf(c1) as c1, udf(c2) as c2 FROM t1)
 -- !query 2 schema
 struct<c1:int,c2:string>
 -- !query 2 output
 -33,12 +33,12  struct<c1:int,c2:string>

 -- !query 3
-SELECT *
-FROM   (SELECT * FROM t1
+SELECT udf(c1) as c1, udf(c2) as c2
+FROM   (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1
         UNION ALL
-        SELECT * FROM t2
+        SELECT udf(c1) as c1, udf(c2) as c2 FROM t2
         UNION ALL
-        SELECT * FROM t2)
+        SELECT udf(c1) as c1, udf(c2) as c2 FROM t2)
 -- !query 3 schema
 struct<c1:decimal(11,1),c2:string>
 -- !query 3 output
 -51,11 +51,11  struct<c1:decimal(11,1),c2:string>

 -- !query 4
-SELECT a
-FROM (SELECT 0 a, 0 b
+SELECT udf(udf(a)) as a
+FROM (SELECT udf(0) a, udf(0) b
       UNION ALL
-      SELECT SUM(1) a, CAST(0 AS BIGINT) b
-      UNION ALL SELECT 0 a, 0 b) T
+      SELECT udf(SUM(1)) a, udf(CAST(0 AS BIGINT)) b
+      UNION ALL SELECT udf(0) a, udf(0) b) T
 -- !query 4 schema
 struct<a:bigint>
 -- !query 4 output
 -89,13 +89,13  struct<>

 -- !query 8
-SELECT 1 AS x,
-       col
-FROM   (SELECT col AS col
-        FROM (SELECT p1.col AS col
+SELECT udf(1) AS x,
+       udf(col) as col
+FROM   (SELECT udf(col) AS col
+        FROM (SELECT udf(p1.col) AS col
               FROM   p1 CROSS JOIN p2
               UNION ALL
-              SELECT col
+              SELECT udf(col)
               FROM p3) T1) T2
 -- !query 8 schema
 struct<x:int,col:int>
 -105,9 +105,9  struct<x:int,col:int>

 -- !query 9
-SELECT map(1, 2), 'str'
+SELECT map(1, 2), udf('str') as str
 UNION ALL
-SELECT map(1, 2, 3, NULL), 1
+SELECT map(1, 2, 3, NULL), udf(1)
 -- !query 9 schema
 struct<map(1, 2):map<int,int>,str:string>
 -- !query 9 output
 -116,9 +116,9  struct<map(1, 2):map<int,int>,str:string>

 -- !query 10
-SELECT array(1, 2), 'str'
+SELECT array(1, 2), udf('str') as str
 UNION ALL
-SELECT array(1, 2, 3, NULL), 1
+SELECT array(1, 2, 3, NULL), udf(1)
 -- !query 10 schema
 struct<array(1, 2):array<int>,str:string>
 -- !query 10 output
```

</p>
</details>

## How was this patch tested?
Tested as guided in SPARK-27921.

Closes #25202 from yiheng/fix_28289.

Authored-by: Yiheng Wang <yihengw@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-26 12:05:45 +09:00
Dongjoon Hyun cefce21acc [MINOR][SQL] Fix log messages of DataWritingSparkTask
## What changes were proposed in this pull request?

This PR fixes the log messages like `attempt 0stage 9.0` by adding a comma followed by space. These are all instances in `DataWritingSparkTask` which was introduced at 6d16b9885d. This should be fixed in `branch-2.4`, too.
```
19/07/25 18:35:01 INFO DataWritingSparkTask: Commit authorized for partition 65 (task 153, attempt 0stage 9.0)
19/07/25 18:35:01 INFO DataWritingSparkTask: Committed partition 65 (task 153, attempt 0stage 9.0)
```

## How was this patch tested?

This only changes log messages. Pass the Jenkins with the existing tests.

Closes #25257 from dongjoon-hyun/DataWritingSparkTask.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-26 09:25:13 +09:00
Ryan Blue 443904a140 [SPARK-27845][SQL] DataSourceV2: InsertTable
## What changes were proposed in this pull request?

Support multiple catalogs in the following InsertTable use cases:

- INSERT INTO [TABLE] catalog.db.tbl
- INSERT OVERWRITE TABLE catalog.db.tbl

Support matrix:

Overwrite|Partitioned Table|Partition Clause |Partition Overwrite Mode|Action
---------|-----------------|-----------------|------------------------|-----
false|*|*|*|AppendData
true|no|(empty)|*|OverwriteByExpression(true)
true|yes|p1,p2 or p1 or p2 or (empty)|STATIC|OverwriteByExpression(true)
true|yes|p2,p2 or p1 or p2 or (empty)|DYNAMIC|OverwritePartitionsDynamic
true|yes|p1=23,p2=3|*|OverwriteByExpression(p1=23 and p2=3)
true|yes|p1=23,p2 or p1=23|STATIC|OverwriteByExpression(p1=23)
true|yes|p1=23,p2 or p1=23|DYNAMIC|OverwritePartitionsDynamic

Notes:
- Assume the partitioned table has 2 partitions: p1 and p2.
- `STATIC` is the default Partition Overwrite Mode for data source tables.
- DSv2 tables currently do not support `IfPartitionNotExists`.

## How was this patch tested?

New tests.
All existing catalyst and sql/core tests.

Closes #24832 from jzhuge/SPARK-27845-pr.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-07-25 15:05:51 -07:00
younggyu chun 89fd2b5efc [SPARK-28288][SQL][PYTHON][TESTS] Convert and port 'window.sql' into UDF test base
## What changes were proposed in this pull request?
This PR adds some tests converted from window.sql to test UDFs. Please see the contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'xxx.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out
index 367dc4f513..9354d5e311 100644
--- a/sql/core/src/test/resources/sql-tests/results/window.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out
 -21,10 +21,10  struct<>

 -- !query 1
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData
-ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY udf(val) ROWS CURRENT ROW) FROM testData
+ORDER BY cate, udf(val)
 -- !query 1 schema
-struct<val:int,cate:string,count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW):bigint>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,count(val) OVER (PARTITION BY cate ORDER BY CAST(udf(cast(val as string)) AS INT) ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW):bigint>
 -- !query 1 output
 NULL   NULL    0
 3      NULL    1
 -38,10 +38,10  NULL        a       0

 -- !query 2
-SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val
-ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, sum(val) OVER(PARTITION BY cate ORDER BY udf(val)
+ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val)
 -- !query 2 schema
-struct<val:int,cate:string,sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING):bigint>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,sum(val) OVER (PARTITION BY cate ORDER BY CAST(udf(cast(val as string)) AS INT) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING):bigint>
 -- !query 2 output
 NULL   NULL    3
 3      NULL    3
 -55,20 +55,20  NULL        a       1

 -- !query 3
-SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long
-ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long
+SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY udf(val_long)
+ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long
 -- !query 3 schema
 struct<>
 -- !query 3 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'bigint' does not match the expected data type 'int'.; line 1 pos 41
+cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'bigint' does not match the expected data type 'int'.; line 1 pos 46

 -- !query 4
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData
-ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE 1 PRECEDING) FROM testData
+ORDER BY cate, udf(val)
 -- !query 4 schema
-struct<val:int,cate:string,count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW):bigint>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,count(val) OVER (PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW):bigint>
 -- !query 4 output
 NULL   NULL    0
 3      NULL    1
 -82,10 +82,10  NULL        a       0

 -- !query 5
-SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val
-RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val
+SELECT val, udf(cate), sum(val) OVER(PARTITION BY udf(cate) ORDER BY val
+RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY udf(cate), val
 -- !query 5 schema
-struct<val:int,cate:string,sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING):bigint>
+struct<val:int,CAST(udf(cast(cate as string)) AS STRING):string,sum(val) OVER (PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING):bigint>
 -- !query 5 output
 NULL   NULL    NULL
 3      NULL    3
 -99,10 +99,10  NULL        a       NULL

 -- !query 6
-SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long
-RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long
+SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY udf(cate) ORDER BY val_long
+RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long
 -- !query 6 schema
-struct<val_long:bigint,cate:string,sum(val_long) OVER (PARTITION BY cate ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING):bigint>
+struct<val_long:bigint,CAST(udf(cast(cate as string)) AS STRING):string,sum(val_long) OVER (PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING):bigint>
 -- !query 6 output
 NULL   NULL    NULL
 1      NULL    1
 -116,10 +116,10  NULL      b       NULL

 -- !query 7
-SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double
-RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double
+SELECT val_double, udf(cate), sum(val_double) OVER(PARTITION BY udf(cate) ORDER BY val_double
+RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY udf(cate), val_double
 -- !query 7 schema
-struct<val_double:double,cate:string,sum(val_double) OVER (PARTITION BY cate ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND CAST(2.5 AS DOUBLE) FOLLOWING):double>
+struct<val_double:double,CAST(udf(cast(cate as string)) AS STRING):string,sum(val_double) OVER (PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND CAST(2.5 AS DOUBLE) FOLLOWING):double>
 -- !query 7 output
 NULL   NULL    NULL
 1.0    NULL    1.0
 -133,10 +133,10  NULL      NULL    NULL

 -- !query 8
-SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date
-RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date
+SELECT val_date, udf(cate), max(val_date) OVER(PARTITION BY udf(cate) ORDER BY val_date
+RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY udf(cate), val_date
 -- !query 8 schema
-struct<val_date:date,cate:string,max(val_date) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING):date>
+struct<val_date:date,CAST(udf(cast(cate as string)) AS STRING):string,max(val_date) OVER (PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING):date>
 -- !query 8 output
 NULL   NULL    NULL
 2017-08-01     NULL    2017-08-01
 -150,11 +150,11  NULL      NULL    NULL

 -- !query 9
-SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp
+SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) ORDER BY val_timestamp
 RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData
-ORDER BY cate, val_timestamp
+ORDER BY udf(cate), val_timestamp
 -- !query 9 schema
-struct<val_timestamp:timestamp,cate:string,avg(CAST(val_timestamp AS DOUBLE)) OVER (PARTITION BY cate ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND interval 3 weeks 2 days 4 hours FOLLOWING):double>
+struct<val_timestamp:timestamp,CAST(udf(cast(cate as string)) AS STRING):string,avg(CAST(val_timestamp AS DOUBLE)) OVER (PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND interval 3 weeks 2 days 4 hours FOLLOWING):double>
 -- !query 9 output
 NULL   NULL    NULL
 2017-07-31 17:00:00    NULL    1.5015456E9
 -168,10 +168,10  NULL      NULL    NULL

 -- !query 10
-SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC
+SELECT val, udf(cate), sum(val) OVER(PARTITION BY cate ORDER BY val DESC
 RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val
 -- !query 10 schema
-struct<val:int,cate:string,sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING):bigint>
+struct<val:int,CAST(udf(cast(cate as string)) AS STRING):string,sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING):bigint>
 -- !query 10 output
 NULL   NULL    NULL
 3      NULL    3
 -185,58 +185,58  NULL      a       NULL

 -- !query 11
-SELECT val, cate, count(val) OVER(PARTITION BY cate
-ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate)
+ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val)
 -- !query 11 schema
 struct<>
 -- !query 11 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 33
+cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 38

 -- !query 12
-SELECT val, cate, count(val) OVER(PARTITION BY cate
-RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate)
+RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val)
 -- !query 12 schema
 struct<>
 -- !query 12 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(PARTITION BY testdata.`cate` RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 33
+cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 38

 -- !query 13
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate
-RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val), cate
+RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val)
 -- !query 13 schema
 struct<>
 -- !query 13 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(PARTITION BY testdata.`cate` ORDER BY testdata.`val` ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 33
+cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY CAST(udf(cast(val as string)) AS INT) ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 38

 -- !query 14
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp
-RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY current_timestamp
+RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val)
 -- !query 14 schema
 struct<>
 -- !query 14 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(PARTITION BY testdata.`cate` ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 33
+cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 38

 -- !query 15
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val
-RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val
+RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val
 -- !query 15 schema
 struct<>
 -- !query 15 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mismatch: The lower bound of a window frame must be less than or equal to the upper bound; line 1 pos 33
+cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mismatch: The lower bound of a window frame must be less than or equal to the upper bound; line 1 pos 38

 -- !query 16
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val
-RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val)
+RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val)
 -- !query 16 schema
 struct<>
 -- !query 16 output
 -245,48 +245,48  org.apache.spark.sql.catalyst.parser.ParseException
 Frame bound value must be a literal.(line 2, pos 30)

 == SQL ==
-SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val
-RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val)
+RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val)
 ------------------------------^^^

 -- !query 17
-SELECT val, cate,
-max(val) OVER w AS max,
-min(val) OVER w AS min,
-min(val) OVER w AS min,
-count(val) OVER w AS count,
-sum(val) OVER w AS sum,
-avg(val) OVER w AS avg,
-stddev(val) OVER w AS stddev,
-first_value(val) OVER w AS first_value,
-first_value(val, true) OVER w AS first_value_ignore_null,
-first_value(val, false) OVER w AS first_value_contain_null,
-last_value(val) OVER w AS last_value,
-last_value(val, true) OVER w AS last_value_ignore_null,
-last_value(val, false) OVER w AS last_value_contain_null,
+SELECT udf(val), cate,
+max(udf(val)) OVER w AS max,
+min(udf(val)) OVER w AS min,
+min(udf(val)) OVER w AS min,
+count(udf(val)) OVER w AS count,
+sum(udf(val)) OVER w AS sum,
+avg(udf(val)) OVER w AS avg,
+stddev(udf(val)) OVER w AS stddev,
+first_value(udf(val)) OVER w AS first_value,
+first_value(udf(val), true) OVER w AS first_value_ignore_null,
+first_value(udf(val), false) OVER w AS first_value_contain_null,
+last_value(udf(val)) OVER w AS last_value,
+last_value(udf(val), true) OVER w AS last_value_ignore_null,
+last_value(udf(val), false) OVER w AS last_value_contain_null,
 rank() OVER w AS rank,
 dense_rank() OVER w AS dense_rank,
 cume_dist() OVER w AS cume_dist,
 percent_rank() OVER w AS percent_rank,
 ntile(2) OVER w AS ntile,
 row_number() OVER w AS row_number,
-var_pop(val) OVER w AS var_pop,
-var_samp(val) OVER w AS var_samp,
-approx_count_distinct(val) OVER w AS approx_count_distinct,
-covar_pop(val, val_long) OVER w AS covar_pop,
-corr(val, val_long) OVER w AS corr,
-stddev_samp(val) OVER w AS stddev_samp,
-stddev_pop(val) OVER w AS stddev_pop,
-collect_list(val) OVER w AS collect_list,
-collect_set(val) OVER w AS collect_set,
-skewness(val_double) OVER w AS skewness,
-kurtosis(val_double) OVER w AS kurtosis
+var_pop(udf(val)) OVER w AS var_pop,
+var_samp(udf(val)) OVER w AS var_samp,
+approx_count_distinct(udf(val)) OVER w AS approx_count_distinct,
+covar_pop(udf(val), udf(val_long)) OVER w AS covar_pop,
+corr(udf(val), udf(val_long)) OVER w AS corr,
+stddev_samp(udf(val)) OVER w AS stddev_samp,
+stddev_pop(udf(val)) OVER w AS stddev_pop,
+collect_list(udf(val)) OVER w AS collect_list,
+collect_set(udf(val)) OVER w AS collect_set,
+skewness(udf(val_double)) OVER w AS skewness,
+kurtosis(udf(val_double)) OVER w AS kurtosis
 FROM testData
-WINDOW w AS (PARTITION BY cate ORDER BY val)
-ORDER BY cate, val
+WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val))
+ORDER BY cate, udf(val)
 -- !query 17 schema
-struct<val:int,cate:string,max:int,min:int,min:int,count:bigint,sum:bigint,avg:double,stddev:double,first_value:int,first_value_ignore_null:int,first_value_contain_null:int,last_value:int,last_value_ignore_null:int,last_value_contain_null:int,rank:int,dense_rank:int,cume_dist:double,percent_rank:double,ntile:int,row_number:int,var_pop:double,var_samp:double,approx_count_distinct:bigint,covar_pop:double,corr:double,stddev_samp:double,stddev_pop:double,collect_list:array<int>,collect_set:array<int>,skewness:double,kurtosis:double>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,max:int,min:int,min:int,count:bigint,sum:bigint,avg:double,stddev:double,first_value:int,first_value_ignore_null:int,first_value_contain_null:int,last_value:int,last_value_ignore_null:int,last_value_contain_null:int,rank:int,dense_rank:int,cume_dist:double,percent_rank:double,ntile:int,row_number:int,var_pop:double,var_samp:double,approx_count_distinct:bigint,covar_pop:double,corr:double,stddev_samp:double,stddev_pop:double,collect_list:array<int>,collect_set:array<int>,skewness:double,kurtosis:double>
 -- !query 17 output
 NULL   NULL    NULL    NULL    NULL    0       NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    1       1       0.5     0.0     1       1       NULL    NULL    0       NULL    NULL
    NULL    NULL    []      []      NULL    NULL
 3      NULL    3       3       3       1       3       3.0     NaN     NULL    3       NULL    3       3       3       2       2       1.0     1.0     2       2       0.0     NaN     1       0.0     NaN
     NaN     0.0     [3]     [3]     NaN     NaN
 -300,9 +300,9  NULL        a       NULL    NULL    NULL    0       NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    NULL    1       1       0.25    0.

 -- !query 18
-SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val
 -- !query 18 schema
-struct<val:int,cate:string,avg(CAST(NULL AS DOUBLE)) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):double>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,avg(CAST(NULL AS DOUBLE)) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):double>
 -- !query 18 output
 NULL   NULL    NULL
 3      NULL    NULL
 -316,7 +316,7  NULL        a       NULL

 -- !query 19
-SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val
+SELECT udf(val), cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, udf(val)
 -- !query 19 schema
 struct<>
 -- !query 19 output
 -325,9 +325,9  Window function row_number() requires window to be ordered, please add ORDER BY

 -- !query 20
-SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val
+SELECT udf(val), cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val
 -- !query 20 schema
-struct<val:int,cate:string,sum(CAST(val AS BIGINT)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):bigint,avg(CAST(val AS BIGINT)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,sum(CAST(val AS BIGINT)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):bigint,avg(CAST(val AS BIGINT)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double>
 -- !query 20 output
 NULL   NULL    13      1.8571428571428572
 3      NULL    13      1.8571428571428572
 -341,7 +341,7  NULL        a       13      1.8571428571428572

 -- !query 21
-SELECT val, cate,
+SELECT udf(val), cate,
 first_value(false) OVER w AS first_value,
 first_value(true, true) OVER w AS first_value_ignore_null,
 first_value(false, false) OVER w AS first_value_contain_null,
 -352,7 +352,7  FROM testData
 WINDOW w AS ()
 ORDER BY cate, val
 -- !query 21 schema
-struct<val:int,cate:string,first_value:boolean,first_value_ignore_null:boolean,first_value_contain_null:boolean,last_value:boolean,last_value_ignore_null:boolean,last_value_contain_null:boolean>
+struct<CAST(udf(cast(val as string)) AS INT):int,cate:string,first_value:boolean,first_value_ignore_null:boolean,first_value_contain_null:boolean,last_value:boolean,last_value_ignore_null:boolean,last_value_contain_null:boolean>
 -- !query 21 output
 NULL   NULL    false   true    false   false   true    false
 3      NULL    false   true    false   false   true    false
 -366,12 +366,12  NULL      a       false   true    false   false   true    false

 -- !query 22
-SELECT cate, sum(val) OVER (w)
+SELECT udf(cate), sum(val) OVER (w)
 FROM testData
 WHERE val is not null
 WINDOW w AS (PARTITION BY cate ORDER BY val)
 -- !query 22 schema
-struct<cate:string,sum(CAST(val AS BIGINT)) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):bigint>
+struct<CAST(udf(cast(cate as string)) AS STRING):string,sum(CAST(val AS BIGINT)) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):bigint>
 -- !query 22 output
 NULL   3
 a      2
```

</p>
</details>

## How was this patch tested?
Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25195 from younggyuchun/master.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-25 22:32:28 +09:00
Gengliang Wang b367b323d2 [SPARK-28497][SQL] Disallow upcasting complex data types to string type
## What changes were proposed in this pull request?

In the current implementation. complex types like Array/Map/StructType are allowed to upcast as StringType.
This is not safe casting. We should disallow it.

## How was this patch tested?

Update the existing test case

Closes #25242 from gengliangwang/fixUpCastStringType.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-25 20:55:01 +09:00
Yuming Wang 045191e610 [SPARK-28293][SQL] Implement Spark's own GetTableTypesOperation
## What changes were proposed in this pull request?

The table type is from Hive now. This will have some issues. For example, we don't support `index_table`, different Hive supports different table types:
Build with Hive 1.2.1:
![image](https://user-images.githubusercontent.com/5399861/60792689-be38b880-a198-11e9-82b8-868992a505e3.png)
Build with Hive 2.3.5:
![image](https://user-images.githubusercontent.com/5399861/60792727-d4467900-a198-11e9-952c-210bb7bb3bed.png)

This pr implement Spark's own `GetTableTypesOperation`.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/60793368-2a67ec00-a19a-11e9-9511-c67483dcc370.png)

Closes #25073 from wangyum/SPARK-28293.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-24 11:27:30 -07:00
shivusondur 167fa0402d [SPARK-28390][SQL][PYTHON][TESTS] Convert and port 'pgSQL/select_having.sql' into UDF test base
## What changes were proposed in this pull request?
changed the test according to steps mentioned in SPARK-27921

<details>
<summary>difference comparing to select_having.sql</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out
index 02536eb..f731d11 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out
 -91,54 +91,54  struct<>

 -- !query 11
-SELECT b, c FROM test_having
-	GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c
+SELECT udf(b), udf(c) FROM test_having
+	GROUP BY b, c HAVING udf(count(*)) = 1 ORDER BY udf(b), udf(c)
 -- !query 11 schema
-struct<b:int,c:string>
+struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(c as string)) AS STRING):string>
 -- !query 11 output
 1	XXXX
 3	bbbb

 -- !query 12
-SELECT b, c FROM test_having
-	GROUP BY b, c HAVING b = 3 ORDER BY b, c
+SELECT udf(b), udf(c) FROM test_having
+	GROUP BY b, c HAVING udf(b) = 3 ORDER BY udf(b), udf(c)
 -- !query 12 schema
-struct<b:int,c:string>
+struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(c as string)) AS STRING):string>
 -- !query 12 output
 3	BBBB
 3	bbbb

 -- !query 13
-SELECT c, max(a) FROM test_having
-	GROUP BY c HAVING count(*) > 2 OR min(a) = max(a)
+SELECT udf(c), max(udf(a)) FROM test_having
+	GROUP BY c HAVING udf(count(*)) > 2 OR udf(min(a)) = udf(max(a))
 	ORDER BY c
 -- !query 13 schema
-struct<c:string,max(a):int>
+struct<CAST(udf(cast(c as string)) AS STRING):string,max(CAST(udf(cast(a as string)) AS INT)):int>
 -- !query 13 output
 XXXX	0
 bbbb	5

 -- !query 14
-SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a)
+SELECT udf(udf(min(udf(a)))), udf(udf(max(udf(a)))) FROM test_having HAVING udf(udf(min(udf(a)))) = udf(udf(max(udf(a))))
 -- !query 14 schema
-struct<min(a):int,max(a):int>
+struct<CAST(udf(cast(cast(udf(cast(min(cast(udf(cast(a as string)) as int)) as string)) as int) as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(max(cast(udf(cast(a as string)) as int)) as string)) as int) as string)) AS INT):int>
 -- !query 14 output

 -- !query 15
-SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a)
+SELECT udf(min(udf(a))), udf(udf(max(a))) FROM test_having HAVING udf(min(a)) < udf(max(udf(a)))
 -- !query 15 schema
-struct<min(a):int,max(a):int>
+struct<CAST(udf(cast(min(cast(udf(cast(a as string)) as int)) as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(max(a) as string)) as int) as string)) AS INT):int>
 -- !query 15 output
 0	9

 -- !query 16
-SELECT a FROM test_having HAVING min(a) < max(a)
+SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a))
 -- !query 16 schema
 struct<>
 -- !query 16 output
 -147,16 +147,16  grouping expressions sequence is empty, and 'default.test_having.`a`' is not an

 -- !query 17
-SELECT 1 AS one FROM test_having HAVING a > 1
+SELECT 1 AS one FROM test_having HAVING udf(a) > 1
 -- !query 17 schema
 struct<>
 -- !query 17 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`a`' given input columns: [one]; line 1 pos 40
+cannot resolve '`a`' given input columns: [one]; line 1 pos 44

 -- !query 18
-SELECT 1 AS one FROM test_having HAVING 1 > 2
+SELECT 1 AS one FROM test_having HAVING udf(udf(1) > udf(2))
 -- !query 18 schema
 struct<one:int>
 -- !query 18 output
 -164,7 +164,7  struct<one:int>

 -- !query 19
-SELECT 1 AS one FROM test_having HAVING 1 < 2
+SELECT 1 AS one FROM test_having HAVING udf(udf(1) < udf(2))
 -- !query 19 schema
 struct<one:int>
 -- !query 19 output
 -172,7 +172,7  struct<one:int>

 -- !query 20
-SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2
+SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2
 -- !query 20 schema
 struct<one:int>
 -- !query 20 output
```
</p>
</details>

## How was this patch tested?
by:

```bash
sudo SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/pgSQL/udf-select_having.sql"
```

Closes #25161 from shivusondur/jira28390.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-24 14:43:39 +09:00
Yuming Wang d67b98ea01 [SPARK-28435][SQL] Support accepting the interval keyword in the schema string
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/7355 add support casting between IntervalType and StringType for scala interface:
```scala
import org.apache.spark.sql.types._
import org.apache.spark.sql.catalyst.expressions._

Cast(Literal("interval 3 month 1 hours"), CalendarIntervalType).eval()
res0: Any = interval 3 months 1 hours
```
But SQL interface does not support it:
```sql
scala> spark.sql("SELECT CAST('interval 3 month 1 hour' AS interval)").show
org.apache.spark.sql.catalyst.parser.ParseException:
DataType interval is not supported.(line 1, pos 41)

== SQL ==
SELECT CAST('interval 3 month 1 hour' AS interval)
-----------------------------------------^^^

  at org.apache.spark.sql.catalyst.parser.AstBuilder.$anonfun$visitPrimitiveDataType$1(AstBuilder.scala:1931)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:108)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitPrimitiveDataType(AstBuilder.scala:1909)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitPrimitiveDataType(AstBuilder.scala:52)
...
```

This PR add supports accepting the `interval` keyword in the schema string. So that SQL interface can support this feature.

## How was this patch tested?

unit tests

Closes #25189 from wangyum/SPARK-28435.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-23 19:40:57 -07:00
HyukjinKwon b83b7927b3 [SPARK-27234][SS][PYTHON] Use InheritableThreadLocal for current epoch in EpochTracker (to support Python UDFs)
## What changes were proposed in this pull request?

This PR proposes to use `InheritableThreadLocal` instead of `ThreadLocal` for current epoch in `EpochTracker`. Python UDF needs threads to write out to and read it from Python processes and when there are new threads, previously set epoch is lost.

After this PR, Python UDFs can be used at Structured Streaming with the continuous mode.

## How was this patch tested?

The test cases were written on the top of https://github.com/apache/spark/pull/24945.
Unit tests were added.

Manual tests.

Closes #24946 from HyukjinKwon/SPARK-27234.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-24 09:59:37 +09:00
Udbhav30 86dad404bd [SPARK-28391][SQL][PYTHON][TESTS] Convert and port 'pgSQL/select_implicit.sql' into UDF test base
## What changes were proposed in this pull request?
This PR adds some tests converted from 'pgSQL/select_implicit.sql' to test UDFs
<details><summary>Diff comparing to 'pgSQL/select_implicit.sql'</summary>
<p>

```diff
... diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out
index 0675820..e6a5995 100755
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out
 -91,9 +91,11  struct<>

 -- !query 11
-SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c
+SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY
+test_missing_target.c
+ORDER BY udf(c)
 -- !query 11 schema
-struct<c:string,count(1):bigint>
+struct<CAST(udf(cast(c as string)) AS STRING):string,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 11 output
 ABAB	2
 BBBB	2
 -104,9 +106,10  cccc	2

 -- !query 12
-SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c
+SELECT udf(count(*)) FROM test_missing_target GROUP BY test_missing_target.c
+ORDER BY udf(c)
 -- !query 12 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 12 output
 2
 2
 -117,18 +120,18  struct<count(1):bigint>

 -- !query 13
-SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b
+SELECT udf(count(*)) FROM test_missing_target GROUP BY a ORDER BY udf(b)
 -- !query 13 schema
 struct<>
 -- !query 13 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`b`' given input columns: [count(1)]; line 1 pos 61
+cannot resolve '`b`' given input columns: [CAST(udf(cast(count(1) as string)) AS BIGINT)]; line 1 pos 70

 -- !query 14
-SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b
+SELECT udf(count(*)) FROM test_missing_target GROUP BY b ORDER BY udf(b)
 -- !query 14 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 14 output
 1
 2
 -137,10 +140,10  struct<count(1):bigint>

 -- !query 15
-SELECT test_missing_target.b, count(*)
-  FROM test_missing_target GROUP BY b ORDER BY b
+SELECT udf(test_missing_target.b), udf(count(*))
+  FROM test_missing_target GROUP BY b ORDER BY udf(b)
 -- !query 15 schema
-struct<b:int,count(1):bigint>
+struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 15 output
 1	1
 2	2
 -149,9 +152,9  struct<b:int,count(1):bigint>

 -- !query 16
-SELECT c FROM test_missing_target ORDER BY a
+SELECT udf(c) FROM test_missing_target ORDER BY udf(a)
 -- !query 16 schema
-struct<c:string>
+struct<CAST(udf(cast(c as string)) AS STRING):string>
 -- !query 16 output
 XXXX
 ABAB
 -166,9 +169,9  CCCC

 -- !query 17
-SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc
+SELECT udf(count(*)) FROM test_missing_target GROUP BY b ORDER BY udf(b) desc
 -- !query 17 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 17 output
 4
 3
 -177,17 +180,17  struct<count(1):bigint>

 -- !query 18
-SELECT count(*) FROM test_missing_target ORDER BY 1 desc
+SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc
 -- !query 18 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 18 output
 10

 -- !query 19
-SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1
+SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1
 -- !query 19 schema
-struct<c:string,count(1):bigint>
+struct<CAST(udf(cast(c as string)) AS STRING):string,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 19 output
 ABAB	2
 BBBB	2
 -198,18 +201,18  cccc	2

 -- !query 20
-SELECT c, count(*) FROM test_missing_target GROUP BY 3
+SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3
 -- !query 20 schema
 struct<>
 -- !query 20 output
 org.apache.spark.sql.AnalysisException
-GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 53
+GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 63

 -- !query 21
-SELECT count(*) FROM test_missing_target x, test_missing_target y
-	WHERE x.a = y.a
-	GROUP BY b ORDER BY b
+SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y
+	WHERE udf(x.a) = udf(y.a)
+	GROUP BY b ORDER BY udf(b)
 -- !query 21 schema
 struct<>
 -- !query 21 output
 -218,10 +221,10  Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10

 -- !query 22
-SELECT a, a FROM test_missing_target
-	ORDER BY a
+SELECT udf(a), udf(a) FROM test_missing_target
+	ORDER BY udf(a)
 -- !query 22 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(a as string)) AS INT):int>
 -- !query 22 output
 0	0
 1	1
 -236,10 +239,10  struct<a:int,a:int>

 -- !query 23
-SELECT a/2, a/2 FROM test_missing_target
-	ORDER BY a/2
+SELECT udf(udf(a)/2), udf(udf(a)/2) FROM test_missing_target
+	ORDER BY udf(udf(a)/2)
 -- !query 23 schema
-struct<(a div 2):int,(a div 2):int>
+struct<CAST(udf(cast((cast(udf(cast(a as string)) as int) div 2) as string)) AS INT):int,CAST(udf(cast((cast(udf(cast(a as string)) as int) div 2) as string)) AS INT):int>
 -- !query 23 output
 0	0
 0	0
 -254,10 +257,10  struct<(a div 2):int,(a div 2):int>

 -- !query 24
-SELECT a/2, a/2 FROM test_missing_target
-	GROUP BY a/2 ORDER BY a/2
+SELECT udf(a/2), udf(a/2) FROM test_missing_target
+	GROUP BY a/2 ORDER BY udf(a/2)
 -- !query 24 schema
-struct<(a div 2):int,(a div 2):int>
+struct<CAST(udf(cast((a div 2) as string)) AS INT):int,CAST(udf(cast((a div 2) as string)) AS INT):int>
 -- !query 24 output
 0	0
 1	1
 -267,11 +270,11  struct<(a div 2):int,(a div 2):int>

 -- !query 25
-SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y
-	WHERE x.a = y.a
-	GROUP BY x.b ORDER BY x.b
+SELECT udf(x.b), udf(count(*)) FROM test_missing_target x, test_missing_target y
+	WHERE udf(x.a) = udf(y.a)
+	GROUP BY x.b ORDER BY udf(x.b)
 -- !query 25 schema
-struct<b:int,count(1):bigint>
+struct<CAST(udf(cast(b as string)) AS INT):int,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 25 output
 1	1
 2	2
 -280,11 +283,11  struct<b:int,count(1):bigint>

 -- !query 26
-SELECT count(*) FROM test_missing_target x, test_missing_target y
-	WHERE x.a = y.a
-	GROUP BY x.b ORDER BY x.b
+SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y
+	WHERE udf(x.a) = udf(y.a)
+	GROUP BY x.b ORDER BY udf(x.b)
 -- !query 26 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 26 output
 1
 2
 -293,22 +296,22  struct<count(1):bigint>

 -- !query 27
-SELECT a%2, count(b) FROM test_missing_target
+SELECT a%2, udf(count(udf(b))) FROM test_missing_target
 GROUP BY test_missing_target.a%2
-ORDER BY test_missing_target.a%2
+ORDER BY udf(test_missing_target.a%2)
 -- !query 27 schema
-struct<(a % 2):int,count(b):bigint>
+struct<(a % 2):int,CAST(udf(cast(count(cast(udf(cast(b as string)) as int)) as string)) AS BIGINT):bigint>
 -- !query 27 output
 0	5
 1	5

 -- !query 28
-SELECT count(c) FROM test_missing_target
+SELECT udf(count(c)) FROM test_missing_target
 GROUP BY lower(test_missing_target.c)
-ORDER BY lower(test_missing_target.c)
+ORDER BY udf(lower(test_missing_target.c))
 -- !query 28 schema
-struct<count(c):bigint>
+struct<CAST(udf(cast(count(c) as string)) AS BIGINT):bigint>
 -- !query 28 output
 2
 3
 -317,18 +320,18  struct<count(c):bigint>

 -- !query 29
-SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b
+SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY a ORDER BY udf(b)
 -- !query 29 schema
 struct<>
 -- !query 29 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`b`' given input columns: [count(a)]; line 1 pos 61
+cannot resolve '`b`' given input columns: [CAST(udf(cast(count(cast(udf(cast(a as string)) as int)) as string)) AS BIGINT)]; line 1 pos 75

 -- !query 30
-SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2
+SELECT udf(count(b)) FROM test_missing_target GROUP BY b/2 ORDER BY udf(b/2)
 -- !query 30 schema
-struct<count(b):bigint>
+struct<CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 30 output
 1
 5
 -336,10 +339,10  struct<count(b):bigint>

 -- !query 31
-SELECT lower(test_missing_target.c), count(c)
-  FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c)
+SELECT udf(lower(test_missing_target.c)), udf(count(udf(c)))
+  FROM test_missing_target GROUP BY lower(c) ORDER BY udf(lower(c))
 -- !query 31 schema
-struct<lower(c):string,count(c):bigint>
+struct<CAST(udf(cast(lower(c) as string)) AS STRING):string,CAST(udf(cast(count(cast(udf(cast(c as string)) as string)) as string)) AS BIGINT):bigint>
 -- !query 31 output
 abab	2
 bbbb	3
 -348,9 +351,9  xxxx	1

 -- !query 32
-SELECT a FROM test_missing_target ORDER BY upper(d)
+SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d)))
 -- !query 32 schema
-struct<a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 32 output
 0
 1
 -365,19 +368,19  struct<a:int>

 -- !query 33
-SELECT count(b) FROM test_missing_target
-	GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc
+SELECT udf(count(b)) FROM test_missing_target
+	GROUP BY (b + 1) / 2 ORDER BY udf((b + 1) / 2) desc
 -- !query 33 schema
-struct<count(b):bigint>
+struct<CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 33 output
 7
 3

 -- !query 34
-SELECT count(x.a) FROM test_missing_target x, test_missing_target y
-	WHERE x.a = y.a
-	GROUP BY b/2 ORDER BY b/2
+SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y
+	WHERE udf(x.a) = udf(y.a)
+	GROUP BY b/2 ORDER BY udf(b/2)
 -- !query 34 schema
 struct<>
 -- !query 34 output
 -386,11 +389,12  Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10

 -- !query 35
-SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y
-	WHERE x.a = y.a
-	GROUP BY x.b/2 ORDER BY x.b/2
+SELECT udf(x.b/2), udf(count(udf(x.b))) FROM test_missing_target x,
+test_missing_target y
+	WHERE udf(x.a) = udf(y.a)
+	GROUP BY x.b/2 ORDER BY udf(x.b/2)
 -- !query 35 schema
-struct<(b div 2):int,count(b):bigint>
+struct<CAST(udf(cast((b div 2) as string)) AS INT):int,CAST(udf(cast(count(cast(udf(cast(b as string)) as int)) as string)) AS BIGINT):bigint>
 -- !query 35 output
 0	1
 1	5
 -398,14 +402,14  struct<(b div 2):int,count(b):bigint>

 -- !query 36
-SELECT count(b) FROM test_missing_target x, test_missing_target y
-	WHERE x.a = y.a
+SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y
+	WHERE udf(x.a) = udf(y.a)
 	GROUP BY x.b/2
 -- !query 36 schema
 struct<>
 -- !query 36 output
 org.apache.spark.sql.AnalysisException
-Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 13
+Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 21

 -- !query 37
```

</p>
</details>

## How was this patch tested?
Tested as Guided in SPARK-27921

Closes #25233 from Udbhav30/master.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-24 09:47:08 +09:00
Douglas R Colkitt 8fc5cb6285 [SPARK-28473][DOC] Stylistic consistency of build command in README
## What changes were proposed in this pull request?

Change the format of the build command in the README to start with a `./` prefix

    ./build/mvn -DskipTests clean package

This increases stylistic consistency across the README- all the other commands have a `./` prefix. Having a visible `./` prefix also makes it clear to the user that the shell command requires the current working directory to be at the repository root.

## How was this patch tested?

README.md was reviewed both in raw markdown and in the Github rendered landing page for stylistic consistency.

Closes #25231 from Mister-Meeseeks/master.

Lead-authored-by: Douglas R Colkitt <douglas.colkitt@gmail.com>
Co-authored-by: Mister-Meeseeks <douglas.colkitt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-23 16:29:46 -07:00
Wenchen Fan a45739d97e [SPARK-28054][SQL][FOLLOWUP] move the bug fix closer to where causes the issue
## What changes were proposed in this pull request?

The bug fixed by https://github.com/apache/spark/pull/24886 is caused by Hive's `loadDynamicPartitions`. It's better to keep the fix surgical and put it right before we call `loadDynamicPartitions`.

This also makes the fix safer, instead of analyzing all the callers of `saveAsHiveFile` and proving that they are safe.

## How was this patch tested?

N/A

Closes #25234 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-23 11:17:43 -07:00
Wenchen Fan e04f696f7f [SPARK-28346][SQL] clone the query plan between analyzer, optimizer and planner
## What changes were proposed in this pull request?

query plan was designed to be immutable, but sometimes we do allow it to carry mutable states, because of the complexity of the SQL system. One example is `TreeNodeTag`. It's a state of `TreeNode` and can be carried over during copy and transform. The adaptive execution framework relies on it to link the logical and physical plans.

This leads to a problem: when we get `QueryExecution#analyzed`, the plan can be changed unexpectedly because it's mutable. I hit a real issue in https://github.com/apache/spark/pull/25107 : I use `TreeNodeTag` to carry dataset id in logical plans. However, the analyzed plan ends up with many duplicated dataset id tags in different nodes. It turns out that, the optimizer transforms the logical plan and add the tag to more nodes.

For example, the logical plan is `SubqueryAlias(Filter(...))`, and I expect only the `SubqueryAlais` has the dataset id tag. However, the optimizer removes `SubqueryAlias` and carries over the dataset id tag to `Filter`. When I go back to the analyzed plan, both `SubqueryAlias` and `Filter` has the dataset id tag, which breaks my assumption.

Since now query plan is mutable, I think it's better to limit the life cycle of a query plan instance. We can clone the query plan between analyzer, optimizer and planner, so that the life cycle is limited in one stage.

## How was this patch tested?

new test

Closes #25111 from cloud-fan/clone.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-23 09:00:39 -07:00
Yuming Wang 022667cea6 [SPARK-28469][SQL] Change CalendarIntervalType's readable string representation from calendarinterval to interval
## What changes were proposed in this pull request?

This PR change `CalendarIntervalType`'s readable string representation from `calendarinterval` to `interval`.

## How was this patch tested?

Existing UT

Closes #25225 from wangyum/SPARK-28469.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-22 20:53:59 -07:00
WeichenXu 185c93e701 [SPARK-28431][SQL] Set maximum error message length in CSV datasource's parsing and writing
## What changes were proposed in this pull request?

Fix CSV datasource to throw `com.univocity.parsers.common.TextParsingException` with large size message, which will make log output consume large disk space.
This issue is troublesome when sometimes we need parse CSV with large size column.

This PR proposes to set CSV parser/writer settings by `setErrorContentLength(1000)` to limit the error message length.

## How was this patch tested?

Manually.

```
val s = "a" * 40 * 1000000
Seq(s).toDF.write.mode("overwrite").csv("/tmp/bogdan/es4196.csv")

spark.read .option("maxCharsPerColumn", 30000000) .csv("/tmp/bogdan/es4196.csv").count
```

**Before:**
The thrown message will include error content of about 30MB size (The column size exceed the max value 30MB, so the error content include the whole parsed content, so it is 30MB).

**After:**
The thrown message will include error content like "...aaa...aa" (the number of 'a' is 1024), i.e. limit the content size to be 1024.

Closes #25184 from WeichenXu123/limit_csv_exception_size.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-23 10:44:59 +09:00
Maxim Gekk 2d74f14d74 [SPARK-28432][SQL] Add make_date function
## What changes were proposed in this pull request?

New function `make_date()` takes 3 columns `year`, `month` and `day`, and makes new column of the `DATE` type. If values in the input columns are `null` or out of valid ranges, the function returns `null`. Valid ranges are:
- `year` - `[1, 9999]`
- `month` - `[1, 12]`
- `day` - `[1, 31]`

Also constructed date must be valid otherwise `make_date` returns `null`.

The function is implemented similarly to `make_date` in PostgreSQL: https://www.postgresql.org/docs/11/functions-datetime.html to maintain feature parity with it.

Here is an example:
```sql
select make_date(2013, 7, 15);
2013-07-15
```

## How was this patch tested?

Added new tests to `DateExpressionsSuite`.

Closes #25210 from MaxGekk/make_date-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-22 15:17:06 -07:00
Stavros Kontopoulos 5b378e6efc [SPARK-28280][SQL][PYTHON][TESTS] Convert and port 'group-by.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `group-by.sql` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).
<details><summary>Diff comparing to 'group-by.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
index 3a5df254f2..0118c05b1d 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
 -13,26 +13,26  struct<>

 -- !query 1
-SELECT a, COUNT(b) FROM testData
+SELECT udf(a), udf(COUNT(b)) FROM testData
 -- !query 1 schema
 struct<>
 -- !query 1 output
 org.apache.spark.sql.AnalysisException
-grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.;
+grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(CAST(udf(cast(count(b) as string)) AS BIGINT) AS `CAST(udf(cast(count(b) as string)) AS BIGINT)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.;

 -- !query 2
-SELECT COUNT(a), COUNT(b) FROM testData
+SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData
 -- !query 2 schema
-struct<count(a):bigint,count(b):bigint>
+struct<count(CAST(udf(cast(a as string)) AS INT)):bigint,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 2 output
 7	7

 -- !query 3
-SELECT a, COUNT(b) FROM testData GROUP BY a
+SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a
 -- !query 3 schema
-struct<a:int,count(b):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 3 output
 1	2
 2	2
 -41,7 +41,7  NULL	1

 -- !query 4
-SELECT a, COUNT(b) FROM testData GROUP BY b
+SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b
 -- !query 4 schema
 struct<>
 -- !query 4 output
 -50,9 +50,9  expression 'testdata.`a`' is neither present in the group by, nor is it an aggre

 -- !query 5
-SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a
+SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a)
 -- !query 5 schema
-struct<count(a):bigint,count(b):bigint>
+struct<count(CAST(udf(cast(a as string)) AS INT)):bigint,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 5 output
 0	1
 2	2
 -61,15 +61,15  struct<count(a):bigint,count(b):bigint>

 -- !query 6
-SELECT 'foo', COUNT(a) FROM testData GROUP BY 1
+SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1
 -- !query 6 schema
-struct<foo:string,count(a):bigint>
+struct<foo:string,count(CAST(udf(cast(a as string)) AS INT)):bigint>
 -- !query 6 output
 foo	7

 -- !query 7
-SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 7 schema
 struct<foo:string>
 -- !query 7 output
 -77,25 +77,25  struct<foo:string>

 -- !query 8
-SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1
 -- !query 8 schema
-struct<foo:string,approx_count_distinct(a):bigint>
+struct<foo:string,CAST(udf(cast(approx_count_distinct(cast(udf(cast(a as string)) as int), 0.05, 0, 0) as string)) AS BIGINT):bigint>
 -- !query 8 output

 -- !query 9
-SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1
 -- !query 9 schema
-struct<foo:string,max(named_struct(a, a)):struct<a:int>>
+struct<foo:string,max(named_struct(col1, CAST(udf(cast(a as string)) AS INT))):struct<col1:int>>
 -- !query 9 output

 -- !query 10
-SELECT a + b, COUNT(b) FROM testData GROUP BY a + b
+SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b
 -- !query 10 schema
-struct<(a + b):int,count(b):bigint>
+struct<CAST(udf(cast((a + b) as string)) AS INT):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 10 output
 2	1
 3	2
 -105,7 +105,7  NULL	1

 -- !query 11
-SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1
+SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1
 -- !query 11 schema
 struct<>
 -- !query 11 output
 -114,37 +114,35  expression 'testdata.`a`' is neither present in the group by, nor is it an aggre

 -- !query 12
-SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1
+SELECT udf(a + 1 + 1), udf(COUNT(b)) FROM testData GROUP BY udf(a + 1)
 -- !query 12 schema
-struct<((a + 1) + 1):int,count(b):bigint>
+struct<>
 -- !query 12 output
-3	2
-4	2
-5	2
-NULL	1
+org.apache.spark.sql.AnalysisException
+expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;

 -- !query 13
-SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a)
+SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a))
 FROM testData
 -- !query 13 schema
-struct<skewness(CAST(a AS DOUBLE)):double,kurtosis(CAST(a AS DOUBLE)):double,min(a):int,max(a):int,avg(a):double,var_samp(CAST(a AS DOUBLE)):double,stddev_samp(CAST(a AS DOUBLE)):double,sum(a):bigint,count(a):bigint>
+struct<skewness(CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double,CAST(udf(cast(kurtosis(cast(a as double)) as string)) AS DOUBLE):double,CAST(udf(cast(min(a) as string)) AS INT):int,max(CAST(udf(cast(a as string)) AS INT)):int,CAST(udf(cast(avg(cast(cast(udf(cast(a as string)) as int) as bigint)) as string)) AS DOUBLE):double,CAST(udf(cast(var_samp(cast(a as double)) as string)) AS DOUBLE):double,stddev_samp(CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double,CAST(udf(cast(sum(cast(a as bigint)) as string)) AS BIGINT):bigint,CAST(udf(cast(count(a) as string)) AS BIGINT):bigint>
 -- !query 13 output
 -0.2723801058145729	-1.5069204152249134	1	3	2.142857142857143	0.8095238095238094	0.8997354108424372	15	7

 -- !query 14
-SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a
+SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a
 -- !query 14 schema
-struct<count(DISTINCT b):bigint,count(DISTINCT b, c):bigint>
+struct<count(DISTINCT CAST(udf(cast(b as string)) AS INT)):bigint,CAST(udf(cast(count(distinct b, c) as string)) AS BIGINT):bigint>
 -- !query 14 output
 1	1

 -- !query 15
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k
+SELECT a AS k, COUNT(udf(b)) FROM testData GROUP BY k
 -- !query 15 schema
-struct<k:int,count(b):bigint>
+struct<k:int,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 15 output
 1	2
 2	2
 -153,21 +151,21  NULL	1

 -- !query 16
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1
+SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1
 -- !query 16 schema
-struct<k:int,count(b):bigint>
+struct<k:int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 16 output
 2	2
 3	2

 -- !query 17
-SELECT COUNT(b) AS k FROM testData GROUP BY k
+SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k
 -- !query 17 schema
 struct<>
 -- !query 17 output
 org.apache.spark.sql.AnalysisException
-aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`);
+aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT);

 -- !query 18
 -180,7 +178,7  struct<>

 -- !query 19
-SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a
+SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY a
 -- !query 19 schema
 struct<>
 -- !query 19 output
 -197,32 +195,32  spark.sql.groupByAliases	false

 -- !query 21
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k
+SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k
 -- !query 21 schema
 struct<>
 -- !query 21 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47
+cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57

 -- !query 22
-SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a
+SELECT a, COUNT(udf(1)) FROM testData WHERE false GROUP BY a
 -- !query 22 schema
-struct<a:int,count(1):bigint>
+struct<a:int,count(CAST(udf(cast(1 as string)) AS INT)):bigint>
 -- !query 22 output

 -- !query 23
-SELECT COUNT(1) FROM testData WHERE false
+SELECT udf(COUNT(1)) FROM testData WHERE false
 -- !query 23 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 23 output
 0

 -- !query 24
-SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t
+SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t
 -- !query 24 schema
 struct<1:int>
 -- !query 24 output
 -232,7 +230,7  struct<1:int>
 -- !query 25
 SELECT 1 from (
   SELECT 1 AS z,
-  MIN(a.x)
+  udf(MIN(a.x))
   FROM (select 1 as x) a
   WHERE false
 ) b
 -244,32 +242,32  struct<1:int>

 -- !query 26
-SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*)
+SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*)
   FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y)
 -- !query 26 schema
-struct<corr(DISTINCT CAST(x AS DOUBLE), CAST(y AS DOUBLE)):double,corr(DISTINCT CAST(y AS DOUBLE), CAST(x AS DOUBLE)):double,count(1):bigint>
+struct<corr(DISTINCT CAST(x AS DOUBLE), CAST(y AS DOUBLE)):double,CAST(udf(cast(corr(distinct cast(y as double), cast(x as double)) as string)) AS DOUBLE):double,count(1):bigint>
 -- !query 26 output
 1.0	1.0	3

 -- !query 27
-SELECT 1 FROM range(10) HAVING true
+SELECT udf(1) FROM range(10) HAVING true
 -- !query 27 schema
-struct<1:int>
+struct<CAST(udf(cast(1 as string)) AS INT):int>
 -- !query 27 output
 1

 -- !query 28
-SELECT 1 FROM range(10) HAVING MAX(id) > 0
+SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0
 -- !query 28 schema
-struct<1:int>
+struct<CAST(udf(cast(cast(udf(cast(1 as string)) as int) as string)) AS INT):int>
 -- !query 28 output
 1

 -- !query 29
-SELECT id FROM range(10) HAVING id > 0
+SELECT udf(id) FROM range(10) HAVING id > 0
 -- !query 29 schema
 struct<>
 -- !query 29 output
 -291,33 +289,33  struct<>

 -- !query 31
-SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0
+SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0
 -- !query 31 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(every(v) as string)) AS BOOLEAN):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 31 output
 NULL	NULL	NULL

 -- !query 32
-SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4
+SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4
 -- !query 32 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(every(cast(udf(cast(v as string)) as boolean)) as string)) AS BOOLEAN):boolean,some(v):boolean,any(v):boolean>
 -- !query 32 output
 NULL	NULL	NULL

 -- !query 33
-SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5
+SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5
 -- !query 33 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<every(v):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 33 output
 false	true	true

 -- !query 34
-SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k
+SELECT k, every(v), udf(some(v)), any(v) FROM test_agg GROUP BY k
 -- !query 34 schema
-struct<k:int,every(v):boolean,some(v):boolean,any(v):boolean>
+struct<k:int,every(v):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 34 output
 1	false	true	true
 2	true	true	true
 -327,9 +325,9  struct<k:int,every(v):boolean,some(v):boolean,any(v):boolean>

 -- !query 35
-SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false
+SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false
 -- !query 35 schema
-struct<k:int,every(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every(v):boolean>
 -- !query 35 output
 1	false
 3	false
 -337,16 +335,16  struct<k:int,every(v):boolean>

 -- !query 36
-SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL
+SELECT k, udf(every(v)) FROM test_agg GROUP BY k HAVING every(v) IS NULL
 -- !query 36 schema
-struct<k:int,every(v):boolean>
+struct<k:int,CAST(udf(cast(every(v) as string)) AS BOOLEAN):boolean>
 -- !query 36 output
 4	NULL

 -- !query 37
 SELECT k,
-       Every(v) AS every
+       udf(Every(v)) AS every
 FROM   test_agg
 WHERE  k = 2
        AND v IN (SELECT Any(v)
 -360,7 +358,7  struct<k:int,every:boolean>

 -- !query 38
-SELECT k,
+SELECT udf(udf(k)),
        Every(v) AS every
 FROM   test_agg
 WHERE  k = 2
 -369,45 +367,45  WHERE  k = 2
                  WHERE  k = 1)
 GROUP  BY k
 -- !query 38 schema
-struct<k:int,every:boolean>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,every:boolean>
 -- !query 38 output

 -- !query 39
-SELECT every(1)
+SELECT every(udf(1))
 -- !query 39 schema
 struct<>
 -- !query 39 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7
+cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7

 -- !query 40
-SELECT some(1S)
+SELECT some(udf(1S))
 -- !query 40 schema
 struct<>
 -- !query 40 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7
+cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7

 -- !query 41
-SELECT any(1L)
+SELECT any(udf(1L))
 -- !query 41 schema
 struct<>
 -- !query 41 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7
+cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7

 -- !query 42
-SELECT every("true")
+SELECT udf(every("true"))
 -- !query 42 schema
 struct<>
 -- !query 42 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7
+cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11

 -- !query 43
 -428,9 +426,9  struct<k:int,v:boolean,every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST

 -- !query 44
-SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
+SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
 -- !query 44 schema
-struct<k:int,v:boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
+struct<k:int,CAST(udf(cast(cast(udf(cast(v as string)) as boolean) as string)) AS BOOLEAN):boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
 -- !query 44 output
 1	false	false
 1	true	true
 -445,9 +443,9  struct<k:int,v:boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST R

 -- !query 45
-SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
+SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
 -- !query 45 schema
-struct<k:int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
 -- !query 45 output
 1	false	false
 1	true	true
 -462,17 +460,17  struct<k:int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RA

 -- !query 46
-SELECT count(*) FROM test_agg HAVING count(*) > 1L
+SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L
 -- !query 46 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 46 output
 10

 -- !query 47
-SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true
+SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true
 -- !query 47 schema
-struct<k:int,max(v):boolean>
+struct<k:int,CAST(udf(cast(max(v) as string)) AS BOOLEAN):boolean>
 -- !query 47 output
 1	true
 2	true
 -480,7 +478,7  struct<k:int,max(v):boolean>

 -- !query 48
-SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L
+SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L
 -- !query 48 schema
 struct<cnt:bigint>
 -- !query 48 output
 -488,7 +486,7  struct<cnt:bigint>

 -- !query 49
-SELECT count(*) FROM test_agg WHERE count(*) > 1L
+SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L
 -- !query 49 schema
 struct<>
 -- !query 49 output
 -500,7 +498,7  Invalid expressions: [count(1)];

 -- !query 50
-SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L
+SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L
 -- !query 50 schema
 struct<>
 -- !query 50 output
 -512,7 +510,7  Invalid expressions: [count(1)];

 -- !query 51
-SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1
+SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1
 -- !query 51 schema
 struct<>
 -- !query 51 output

```

</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).
Verified pandas & pyarrow versions:
```$python3
Python 3.6.8 (default, Jan 14 2019, 11:02:34)
[GCC 8.0.1 20180414 (experimental) [trunk revision 259383]] on linux
Type "help", "copyright", "credits" or "license" for more information.
>>> import pandas
>>> import pyarrow
>>> pyarrow.__version__
'0.14.0'
>>> pandas.__version__
'0.24.2'
```
From the sql output it seems that sql statements are evaluated correctly given that udf returns a string and may change results as Null will be returned as None and will be counted in returned values.

Closes #25098 from skonto/group-by.sql.

Authored-by: Stavros Kontopoulos <st.kontopoulos@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-22 22:41:42 +09:00
Shixiong Zhu 62e28248f1 [SPARK-28456][SQL] Add a public API Encoder.makeCopy to allow creating Encoder without touching Scala Reflection
## What changes were proposed in this pull request?

Because `Encoder` is not thread safe, the user cannot reuse an `Encoder` in multiple `Dataset`s. However, creating an `Encoder` for a complicated class is slow due to Scala Reflection. To eliminate the cost of Scala Reflection, right now I usually use the private API `ExpressionEncoder.copy` as follows:

```scala
object FooEncoder {
  private lazy val _encoder: ExpressionEncoder[Foo] = ExpressionEncoder[Foo]()
  implicit def encoder: ExpressionEncoder[Foo] = _encoder.copy()
}
```

This PR proposes a new method `makeCopy` in `Encoder` so that the above codes can be rewritten using public APIs.

```scala
object FooEncoder {
  private lazy val _encoder: Encoder[Foo] = Encoders.product[Foo]()
  implicit def encoder: Encoder[Foo] = _encoder.makeCopy
}
```

The method name is consistent with `TreeNode.makeCopy`.

## How was this patch tested?

Jenkins

Closes #25209 from zsxwing/encoder-copy.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-22 12:31:51 +08:00
mcheah 7ed0088539 [SPARK-27724][SQL] Implement REPLACE TABLE and REPLACE TABLE AS SELECT with V2
## What changes were proposed in this pull request?

Implements the `REPLACE TABLE` and `REPLACE TABLE AS SELECT` logical plans. `REPLACE TABLE` is now a valid operation in spark-sql provided that the tables being modified are managed by V2 catalogs.

This also introduces an atomic mix-in that table catalogs can choose to implement. Table catalogs can now implement `TransactionalTableCatalog`. The semantics of this API are that table creation and replacement can be "staged" and then "committed".

On the execution of `REPLACE TABLE AS SELECT`, `REPLACE TABLE`, and `CREATE TABLE AS SELECT`, if the catalog implements transactional operations, the physical plan will use said functionality. Otherwise, these operations fall back on non-atomic variants. For `REPLACE TABLE` in particular, the usage of non-atomic operations can unfortunately lead to inconsistent state.

## How was this patch tested?

Unit tests - multiple additions to `DataSourceV2SQLSuite`.

Closes #24798 from mccheah/spark-27724.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-22 12:08:46 +08:00
Marco Gaido a783690d8a [SPARK-28369][SQL] Honor spark.sql.decimalOperations.nullOnOverflow in ScalaUDF result
## What changes were proposed in this pull request?

When a `ScalaUDF` returns a value which overflows, currently it returns null regardless of the value of the config `spark.sql.decimalOperations.nullOnOverflow`.

The PR makes it respect the above-mentioned config and behave accordingly.

## How was this patch tested?

added UT

Closes #25144 from mgaido91/SPARK-28369.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-22 10:39:40 +08:00
Takeshi Yamamuro fced6696a7 [SPARK-28462][SQL][TEST] Add a prefix '*' to non-nullable attribute names in PlanTestBase.comparePlans failures
## What changes were proposed in this pull request?
This pr proposes to add a prefix '*' to non-nullable attribute names in PlanTestBase.comparePlans failures. In the current master, nullability mismatches might generate the same error message for left/right logical plans like this;
```
// This failure message was extracted from #24765
- constraints should be inferred from aliased literals *** FAILED ***
  == FAIL: Plans do not match ===
  !'Join Inner, (two#0 = a#0)                    'Join Inner, (two#0 = a#0)
   :- Filter (isnotnull(a#0) AND (2 <=> a#0))     :- Filter (isnotnull(a#0) AND (2 <=> a#0))
   :  +- LocalRelation <empty>, [a#0, b#0, c#0]   :  +- LocalRelation <empty>, [a#0, b#0, c#0]
   +- Project [2 AS two#0]                        +- Project [2 AS two#0]
      +- LocalRelation <empty>, [a#0, b#0, c#0]      +- LocalRelation <empty>, [a#0, b#0, c#0] (PlanTest.scala:145)
```
With this pr, this error message is changed to one below;
```
- constraints should be inferred from aliased literals *** FAILED ***
  == FAIL: Plans do not match ===
  !'Join Inner, (*two#0 = a#0)                    'Join Inner, (*two#0 = *a#0)
   :- Filter (isnotnull(a#0) AND (2 <=> a#0))     :- Filter (isnotnull(a#0) AND (2 <=> a#0))
   :  +- LocalRelation <empty>, [a#0, b#0, c#0]   :  +- LocalRelation <empty>, [a#0, b#0, c#0]
   +- Project [2 AS two#0]                        +- Project [2 AS two#0]
      +- LocalRelation <empty>, [a#0, b#0, c#0]      +- LocalRelation <empty>, [a#0, b#0, c#0] (PlanTest.scala:145)
```

## How was this patch tested?
N/A

Closes #25213 from maropu/MarkForNullability.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-21 13:34:35 -07:00
Takeshi Yamamuro 6e65d39576 [SPARK-28189][SQL][FOLLOW-UP] Remove the unnecessary test in DataFrameSuite
## What changes were proposed in this pull request?
This pr is to remove the unnecessary test  in DataFrameSuite.

## How was this patch tested?
N/A

Closes #25216 from maropu/SPARK-28189-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-21 00:07:35 -07:00
Xingbo Jiang 36d7d81d23 [SPARK-27815][SQL][FOLLOWUP][DOC] Update comment that references PushDownPredicate
## What changes were proposed in this pull request?

The optimize rule `PushDownPredicate` has been combined into `PushDownPredicates`, update the comment that references the old rule.

## How was this patch tested?

N/A

Closes #25207 from jiangxb1987/comment.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-20 16:44:28 +09:00
Terry Kim 771616eac9 [SPARK-28282][SQL][PYTHON][TESTS] Convert and port 'inline-table.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `inline-table.sql` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'inline-table.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out
index 4e80f0bda5..2cf24e50c8 100644
--- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out
 -3,33 +3,33

 -- !query 0
-select * from values ("one", 1)
+select udf(col1), udf(col2) from values ("one", 1)
 -- !query 0 schema
-struct<col1:string,col2:int>
+struct<CAST(udf(cast(col1 as string)) AS STRING):string,CAST(udf(cast(col2 as string)) AS INT):int>
 -- !query 0 output
 one	1

 -- !query 1
-select * from values ("one", 1) as data
+select udf(col1), udf(udf(col2)) from values ("one", 1) as data
 -- !query 1 schema
-struct<col1:string,col2:int>
+struct<CAST(udf(cast(col1 as string)) AS STRING):string,CAST(udf(cast(cast(udf(cast(col2 as string)) as int) as string)) AS INT):int>
 -- !query 1 output
 one	1

 -- !query 2
-select * from values ("one", 1) as data(a, b)
+select udf(a), b from values ("one", 1) as data(a, b)
 -- !query 2 schema
-struct<a:string,b:int>
+struct<CAST(udf(cast(a as string)) AS STRING):string,b:int>
 -- !query 2 output
 one	1

 -- !query 3
-select * from values 1, 2, 3 as data(a)
+select udf(a) from values 1, 2, 3 as data(a)
 -- !query 3 schema
-struct<a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 3 output
 1
 2
 -37,9 +37,9  struct<a:int>

 -- !query 4
-select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b)
+select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b)
 -- !query 4 schema
-struct<a:string,b:int>
+struct<CAST(udf(cast(a as string)) AS STRING):string,b:int>
 -- !query 4 output
 one	1
 three	NULL
 -47,107 +47,107  two	2

 -- !query 5
-select * from values ("one", null), ("two", null) as data(a, b)
+select udf(a), b from values ("one", null), ("two", null) as data(a, b)
 -- !query 5 schema
-struct<a:string,b:null>
+struct<CAST(udf(cast(a as string)) AS STRING):string,b:null>
 -- !query 5 output
 one	NULL
 two	NULL

 -- !query 6
-select * from values ("one", 1), ("two", 2L) as data(a, b)
+select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b)
 -- !query 6 schema
-struct<a:string,b:bigint>
+struct<CAST(udf(cast(a as string)) AS STRING):string,b:bigint>
 -- !query 6 output
 one	1
 two	2

 -- !query 7
-select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b)
+select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b)
 -- !query 7 schema
-struct<a:string,b:bigint>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as string) as string)) AS STRING):string,CAST(udf(cast(b as string)) AS BIGINT):bigint>
 -- !query 7 output
 one	1
 two	4

 -- !query 8
-select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b)
+select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b)
 -- !query 8 schema
-struct<a:string,b:array<int>>
+struct<CAST(udf(cast(a as string)) AS STRING):string,b:array<int>>
 -- !query 8 output
 one	[0,1]
 two	[2,3]

 -- !query 9
-select * from values ("one", 2.0), ("two", 3.0D) as data(a, b)
+select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b)
 -- !query 9 schema
-struct<a:string,b:double>
+struct<CAST(udf(cast(a as string)) AS STRING):string,b:double>
 -- !query 9 output
 one	2.0
 two	3.0

 -- !query 10
-select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b)
+select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b)
 -- !query 10 schema
 struct<>
 -- !query 10 output
 org.apache.spark.sql.AnalysisException
-cannot evaluate expression rand(5) in inline table definition; line 1 pos 29
+cannot evaluate expression rand(5) in inline table definition; line 1 pos 37

 -- !query 11
-select * from values ("one", 2.0), ("two") as data(a, b)
+select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b)
 -- !query 11 schema
 struct<>
 -- !query 11 output
 org.apache.spark.sql.AnalysisException
-expected 2 columns but found 1 columns in row 1; line 1 pos 14
+expected 2 columns but found 1 columns in row 1; line 1 pos 27

 -- !query 12
-select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b)
+select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b)
 -- !query 12 schema
 struct<>
 -- !query 12 output
 org.apache.spark.sql.AnalysisException
-incompatible types found in column b for inline table; line 1 pos 14
+incompatible types found in column b for inline table; line 1 pos 27

 -- !query 13
-select * from values ("one"), ("two") as data(a, b)
+select udf(a), udf(b) from values ("one"), ("two") as data(a, b)
 -- !query 13 schema
 struct<>
 -- !query 13 output
 org.apache.spark.sql.AnalysisException
-expected 2 columns but found 1 columns in row 0; line 1 pos 14
+expected 2 columns but found 1 columns in row 0; line 1 pos 27

 -- !query 14
-select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b)
+select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b)
 -- !query 14 schema
 struct<>
 -- !query 14 output
 org.apache.spark.sql.AnalysisException
-Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 29
+Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 42

 -- !query 15
-select * from values ("one", count(1)), ("two", 2) as data(a, b)
+select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b)
 -- !query 15 schema
 struct<>
 -- !query 15 output
 org.apache.spark.sql.AnalysisException
-cannot evaluate expression count(1) in inline table definition; line 1 pos 29
+cannot evaluate expression count(1) in inline table definition; line 1 pos 42

 -- !query 16
-select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b)
+select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b)
 -- !query 16 schema
-struct<a:timestamp,b:array<timestamp>>
+struct<CAST(udf(cast(a as string)) AS TIMESTAMP):timestamp,b:array<timestamp>>
 -- !query 16 output
 1991-12-06 00:00:00	[1991-12-06 01:00:00.0,1991-12-06 12:00:00.0]

```
</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25124 from imback82/inline-table-sql.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-20 15:21:28 +09:00
Stavros Kontopoulos 9e5e511ca0 [SPARK-28279][SQL][PYTHON][TESTS] Convert and port 'group-analytics.sql' into UDF test base
## What changes were proposed in this pull request?
This PR adds some tests converted from group-analytics.sql to test UDFs. Please see contribution guide of this umbrella ticket - SPARK-27921.

<details><summary>Diff comparing to 'group-analytics.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
index 31e9e08e2c..3439a05727 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
 -13,9 +13,9  struct<>

 -- !query 1
-SELECT a + b, b, udf(SUM(a - b)) FROM testData GROUP BY a + b, b WITH CUBE
+SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE
 -- !query 1 schema
-struct<(a + b):int,b:int,CAST(udf(cast(sum(cast((a - b) as bigint)) as string)) AS BIGINT):bigint>
+struct<(a + b):int,b:int,sum((a - b)):bigint>
 -- !query 1 output
 2	1	0
 2	NULL	0
 -33,9 +33,9  NULL	NULL	3

 -- !query 2
-SELECT a, udf(b), SUM(b) FROM testData GROUP BY a, b WITH CUBE
+SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE
 -- !query 2 schema
-struct<a:int,CAST(udf(cast(b as string)) AS INT):int,sum(b):bigint>
+struct<a:int,b:int,sum(b):bigint>
 -- !query 2 output
 1	1	1
 1	2	2
 -52,9 +52,9  NULL	NULL	9

 -- !query 3
-SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP
+SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP
 -- !query 3 schema
-struct<CAST(udf(cast((a + b) as string)) AS INT):int,b:int,sum((a - b)):bigint>
+struct<(a + b):int,b:int,sum((a - b)):bigint>
 -- !query 3 output
 2	1	0
 2	NULL	0
 -70,9 +70,9  NULL	NULL	3

 -- !query 4
-SELECT a, b, udf(SUM(b)) FROM testData GROUP BY a, b WITH ROLLUP
+SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP
 -- !query 4 schema
-struct<a:int,b:int,CAST(udf(cast(sum(cast(b as bigint)) as string)) AS BIGINT):bigint>
+struct<a:int,b:int,sum(b):bigint>
 -- !query 4 output
 1	1	1
 1	2	2
 -97,7 +97,7  struct<>

 -- !query 6
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year
 -- !query 6 schema
 struct<course:string,year:int,sum(earnings):bigint>
 -- !query 6 output
 -111,7 +111,7  dotNET	2013	48000

 -- !query 7
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year)
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year
 -- !query 7 schema
 struct<course:string,year:int,sum(earnings):bigint>
 -- !query 7 output
 -127,9 +127,9  dotNET	2013	48000

 -- !query 8
-SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year)
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year)
 -- !query 8 schema
-struct<course:string,CAST(udf(cast(year as string)) AS INT):int,sum(earnings):bigint>
+struct<course:string,year:int,sum(earnings):bigint>
 -- !query 8 output
 Java	NULL	50000
 NULL	2012	35000
 -138,26 +138,26  dotNET	NULL	63000

 -- !query 9
-SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course)
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course)
 -- !query 9 schema
-struct<course:string,year:int,CAST(udf(cast(sum(cast(earnings as bigint)) as string)) AS BIGINT):bigint>
+struct<course:string,year:int,sum(earnings):bigint>
 -- !query 9 output
 Java	NULL	50000
 dotNET	NULL	63000

 -- !query 10
-SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year)
+SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year)
 -- !query 10 schema
-struct<CAST(udf(cast(course as string)) AS STRING):string,year:int,sum(earnings):bigint>
+struct<course:string,year:int,sum(earnings):bigint>
 -- !query 10 output
 NULL	2012	35000
 NULL	2013	78000

 -- !query 11
-SELECT course, udf(SUM(earnings)) AS sum FROM courseSales
-GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum)
+SELECT course, SUM(earnings) AS sum FROM courseSales
+GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum
 -- !query 11 schema
 struct<course:string,sum:bigint>
 -- !query 11 output
 -173,7 +173,7  dotNET	63000

 -- !query 12
 SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales
-GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum
+GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum
 -- !query 12 schema
 struct<course:string,sum:bigint,grouping_id(course, earnings):int>
 -- !query 12 output
 -188,10 +188,10  dotNET	63000	1

 -- !query 13
-SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales
+SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales
 GROUP BY CUBE(course, year)
 -- !query 13 schema
-struct<CAST(udf(cast(course as string)) AS STRING):string,CAST(udf(cast(year as string)) AS INT):int,grouping(course):tinyint,grouping(year):tinyint,grouping_id(course, year):int>
+struct<course:string,year:int,grouping(course):tinyint,grouping(year):tinyint,grouping_id(course, year):int>
 -- !query 13 output
 Java	2012	0	0	0
 Java	2013	0	0	0
 -205,7 +205,7  dotNET	NULL	0	1	1

 -- !query 14
-SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, year
+SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year
 -- !query 14 schema
 struct<>
 -- !query 14 output
 -214,7 +214,7  grouping() can only be used with GroupingSets/Cube/Rollup;

 -- !query 15
-SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY course, year
+SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year
 -- !query 15 schema
 struct<>
 -- !query 15 output
 -223,7 +223,7  grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 16
-SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year)
+SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year
 -- !query 16 schema
 struct<course:string,year:int,grouping__id:int>
 -- !query 16 output
 -240,7 +240,7  NULL	NULL	3

 -- !query 17
 SELECT course, year FROM courseSales GROUP BY CUBE(course, year)
-HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year)
+HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, year
 -- !query 17 schema
 struct<course:string,year:int>
 -- !query 17 output
 -250,7 +250,7  dotNET	NULL

 -- !query 18
-SELECT course, udf(year) FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0
+SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0
 -- !query 18 schema
 struct<>
 -- !query 18 output
 -259,7 +259,7  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 19
-SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0
+SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0
 -- !query 19 schema
 struct<>
 -- !query 19 output
 -268,9 +268,9  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 20
-SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0
+SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0
 -- !query 20 schema
-struct<CAST(udf(cast(course as string)) AS STRING):string,year:int>
+struct<course:string,year:int>
 -- !query 20 output
 Java	NULL
 NULL	2012
 -281,7 +281,7  dotNET	NULL

 -- !query 21
 SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year)
-ORDER BY GROUPING(course), GROUPING(year), course, udf(year)
+ORDER BY GROUPING(course), GROUPING(year), course, year
 -- !query 21 schema
 struct<course:string,year:int,grouping(course):tinyint,grouping(year):tinyint>
 -- !query 21 output
 -298,7 +298,7  NULL	NULL	1	1

 -- !query 22
 SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year)
-ORDER BY GROUPING(course), GROUPING(year), course, udf(year)
+ORDER BY GROUPING(course), GROUPING(year), course, year
 -- !query 22 schema
 struct<course:string,year:int,grouping_id(course, year):int>
 -- !query 22 output
 -314,7 +314,7  NULL	NULL	3

 -- !query 23
-SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course)
+SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course)
 -- !query 23 schema
 struct<>
 -- !query 23 output
 -323,7 +323,7  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 24
-SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course)
+SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course)
 -- !query 24 schema
 struct<>
 -- !query 24 output
 -332,7 +332,7  grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup;

 -- !query 25
-SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year
+SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year
 -- !query 25 schema
 struct<course:string,year:int>
 -- !query 25 output
 -348,7 +348,7  NULL	NULL

 -- !query 26
-SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2)
+SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2)
 -- !query 26 schema
 struct<k1:int,k2:int,sum((a - b)):bigint>
 -- !query 26 output
 -368,7 +368,7  NULL	NULL	3

 -- !query 27
-SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b)
+SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b)
 -- !query 27 schema
 struct<k:int,b:int,sum((a - b)):bigint>
 -- !query 27 output
 -386,9 +386,9  NULL	NULL	3

 -- !query 28
-SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k)
+SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k)
 -- !query 28 schema
-struct<CAST(udf(cast((a + b) as string)) AS INT):int,k:int,sum((a - b)):bigint>
+struct<(a + b):int,k:int,sum((a - b)):bigint>
 -- !query 28 output
 NULL	1	3
 NULL	2	0

```

</p>
</details>

## How was this patch tested?

Tested as guided in SPARK-27921.
Verified pandas & pyarrow versions:
```$python3
Python 3.6.8 (default, Jan 14 2019, 11:02:34)
[GCC 8.0.1 20180414 (experimental) [trunk revision 259383]] on linux
Type "help", "copyright", "credits" or "license" for more information.
>>> import pandas
>>> import pyarrow
>>> pyarrow.__version__
'0.14.0'
>>> pandas.__version__
'0.24.2'
```
From the sql output it seems that sql statements are evaluated correctly given that udf returns a string and may change results as Null will be returned as None and will be counted in returned values.

Closes #25196 from skonto/group-analytics.sql.

Authored-by: Stavros Kontopoulos <st.kontopoulos@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-20 15:19:57 +09:00
huangtianhua aeec6a7b28 [SPARK-28433][SQL][TEST] Remove hardware-dependent 0.0/0.0 and NaN comparison assertions
## What changes were proposed in this pull request?

This PR removes a few hardware-dependent assertions which can cause a failure in `aarch64`.

**x86_64**
```
rootdonotdel-openlab-allinone-l00242678:/home/ubuntu# uname -a
Linux donotdel-openlab-allinone-l00242678 4.4.0-154-generic #181-Ubuntu SMP Tue Jun 25 05:29:03 UTC
2019 x86_64 x86_64 x86_64 GNU/Linux

scala> import java.lang.Float.floatToRawIntBits
import java.lang.Float.floatToRawIntBits
scala> floatToRawIntBits(0.0f/0.0f)
res0: Int = -4194304
scala> floatToRawIntBits(Float.NaN)
res1: Int = 2143289344
```

**aarch64**
```
[rootarm-huangtianhua spark]# uname -a
Linux arm-huangtianhua 4.14.0-49.el7a.aarch64 #1 SMP Tue Apr 10 17:22:26 UTC 2018 aarch64 aarch64 aarch64 GNU/Linux

scala> import java.lang.Float.floatToRawIntBits
import java.lang.Float.floatToRawIntBits
scala> floatToRawIntBits(0.0f/0.0f)
res1: Int = 2143289344
scala> floatToRawIntBits(Float.NaN)
res2: Int = 2143289344
```

## How was this patch tested?

Pass the Jenkins (This removes the test coverage).

Closes #25186 from huangtianhua/special-test-case-for-aarch64.

Authored-by: huangtianhua <huangtianhua@huawei.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-19 16:02:12 -07:00
Jungtaek Lim (HeartSaVioR) 4196d7bd34 [SPARK-28199][SS][FOLLOWUP] Remove unnecessary annotations for private API
## What changes were proposed in this pull request?

SPARK-28199 (#24996) hid implementations of Triggers into `private[sql]` and encourage end users to use `Trigger.xxx` methods instead.

As I got some post review comment on 7548a8826d (r34366934) we could remove annotations which are meant to be used with public API.

## How was this patch tested?

N/A

Closes #25200 from HeartSaVioR/SPARK-28199-FOLLOWUP.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-19 08:26:42 -07:00
Terry Kim 453cbf3dd8 [SPARK-28284][SQL][PYTHON][TESTS] Convert and port 'join-empty-relation.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `join-empty-relation.sql` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'join-empty-relation.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out
index 857073a827..e79d01fb14 100644
--- a/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out
 -27,111 +27,111  struct<>

 -- !query 3
-SELECT * FROM t1 INNER JOIN empty_table
+SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a)))
 -- !query 3 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(a as string)) AS INT):int>
 -- !query 3 output

 -- !query 4
-SELECT * FROM t1 CROSS JOIN empty_table
+SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a))
 -- !query 4 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 4 output

 -- !query 5
-SELECT * FROM t1 LEFT OUTER JOIN empty_table
+SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a))
 -- !query 5 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int,a:int>
 -- !query 5 output
 1	NULL

 -- !query 6
-SELECT * FROM t1 RIGHT OUTER JOIN empty_table
+SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a))
 -- !query 6 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(a as string)) AS INT):int>
 -- !query 6 output

 -- !query 7
-SELECT * FROM t1 FULL OUTER JOIN empty_table
+SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a))
 -- !query 7 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,a:int>
 -- !query 7 output
 1	NULL

 -- !query 8
-SELECT * FROM t1 LEFT SEMI JOIN empty_table
+SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a)))
 -- !query 8 schema
-struct<a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 8 output

 -- !query 9
-SELECT * FROM t1 LEFT ANTI JOIN empty_table
+SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a))
 -- !query 9 schema
-struct<a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 9 output
 1

 -- !query 10
-SELECT * FROM empty_table INNER JOIN t1
+SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a))
 -- !query 10 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(a as string)) AS INT):int>
 -- !query 10 output

 -- !query 11
-SELECT * FROM empty_table CROSS JOIN t1
+SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a)))
 -- !query 11 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int,CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 11 output

 -- !query 12
-SELECT * FROM empty_table LEFT OUTER JOIN t1
+SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a))
 -- !query 12 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int,CAST(udf(cast(a as string)) AS INT):int>
 -- !query 12 output

 -- !query 13
-SELECT * FROM empty_table RIGHT OUTER JOIN t1
+SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a))
 -- !query 13 schema
-struct<a:int,a:int>
+struct<a:int,CAST(udf(cast(a as string)) AS INT):int>
 -- !query 13 output
 NULL	1

 -- !query 14
-SELECT * FROM empty_table FULL OUTER JOIN t1
+SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a))
 -- !query 14 schema
-struct<a:int,a:int>
+struct<a:int,CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 14 output
 NULL	1

 -- !query 15
-SELECT * FROM empty_table LEFT SEMI JOIN t1
+SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a)))
 -- !query 15 schema
-struct<a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 15 output

 -- !query 16
-SELECT * FROM empty_table LEFT ANTI JOIN t1
+SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a))
 -- !query 16 schema
 struct<a:int>
 -- !query 16 output
 -139,56 +139,56  struct<a:int>

 -- !query 17
-SELECT * FROM empty_table INNER JOIN empty_table
+SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a)))
 -- !query 17 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 17 output

 -- !query 18
-SELECT * FROM empty_table CROSS JOIN empty_table
+SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a))
 -- !query 18 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 18 output

 -- !query 19
-SELECT * FROM empty_table LEFT OUTER JOIN empty_table
+SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a))
 -- !query 19 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 19 output

 -- !query 20
-SELECT * FROM empty_table RIGHT OUTER JOIN empty_table
+SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a)))
 -- !query 20 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 20 output

 -- !query 21
-SELECT * FROM empty_table FULL OUTER JOIN empty_table
+SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a))
 -- !query 21 schema
-struct<a:int,a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 21 output

 -- !query 22
-SELECT * FROM empty_table LEFT SEMI JOIN empty_table
+SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a))
 -- !query 22 schema
-struct<a:int>
+struct<CAST(udf(cast(cast(udf(cast(a as string)) as int) as string)) AS INT):int>
 -- !query 22 output

 -- !query 23
-SELECT * FROM empty_table LEFT ANTI JOIN empty_table
+SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a))
 -- !query 23 schema
-struct<a:int>
+struct<CAST(udf(cast(a as string)) AS INT):int>
 -- !query 23 output

```
</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25127 from imback82/join-empty-relation-sql.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-19 16:27:26 +09:00
Liang-Chi Hsieh 127bc899ae [SPARK-27707][SQL] Prune unnecessary nested fields from Generate
## What changes were proposed in this pull request?

Performance issue using explode was found when a complex field contains huge array is to get duplicated as the number of exploded array elements. Given example:

```scala
val df = spark.sparkContext.parallelize(Seq(("1",
  Array.fill(M)({
    val i = math.random
    (i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString)
  })))).toDF("col", "arr")
  .selectExpr("col", "struct(col, arr) as st")
  .selectExpr("col", "st.col as col1", "explode(st.arr) as arr_col")
```

The explode causes `st` to be duplicated as many as the exploded elements.

Benchmarks it:

```
[info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4
[info] Intel(R) Core(TM) i7-8750H CPU  2.20GHz
[info] generate big nested struct array:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] generate big nested struct array wholestage off          52668          53162         699          0.0      877803.4       1.0X
[info] generate big nested struct array wholestage on          47261          49093        1125          0.0      787690.2       1.1X
[info]
```

The query plan:
```
== Physical Plan ==
 Project [col#508, st#512.col AS col1#515, arr_col#519]
 +- Generate explode(st#512.arr), [col#508, st#512], false, [arr_col#519]
    +- Project [_1#503 AS col#508, named_struct(col, _1#503, arr, _2#504) AS st#512]
       +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#503, mapobjects(MapObjects_loopValue84, MapObjects_loopIsNull84,      ObjectType(class scala.Tuple4), if (isnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true)))     null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._2, true, false), _3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String,     StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._3, true,  false), _4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84,   MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._4, true, false)), knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, None) AS _2#504]
          +- Scan[obj#534]
```

This patch takes nested column pruning approach to prune unnecessary nested fields. It adds a projection of the needed nested fields as aliases on the child of `Generate`, and substitutes them by alias attributes on the projection on top of `Generate`.

Benchmarks it after the change:
```
 [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4
 [info] Intel(R) Core(TM) i7-8750H CPU  2.20GHz
 [info] generate big nested struct array:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 [info] ------------------------------------------------------------------------------------------------------------------------
 [info] generate big nested struct array wholestage off            311            331          28          0.2        5188.6       1.0X
 [info] generate big nested struct array wholestage on            297            312          15          0.2        4947.3       1.0X
 [info]
```

The query plan:
```
== Physical Plan ==
 Project [col#592, _gen_alias_608#608 AS col1#599, arr_col#603]
 +- Generate explode(st#596.arr), [col#592, _gen_alias_608#608], false, [arr_col#603]
    +- Project [_1#587 AS col#592, named_struct(col, _1#587, arr, _2#588) AS st#596, _1#587 AS _gen_alias_608#608]
       +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(in
 put[0, scala.Tuple2, true]))._1, true, false) AS _1#587, mapobjects(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4),
 if (isnull(lambdavariable(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))) null else named_struct(_1,        staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue102,              MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String,    StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._2,      true, false), _3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString,                                                 knownnotnull(lambdavariable(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._3, true, false), _4,            staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue102,              MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._4, true, false)), knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2,      None) AS _2#588]
          +- Scan[obj#586]
```

This behavior is controlled by a SQL config `spark.sql.optimizer.expression.nestedPruning.enabled`.

## How was this patch tested?

Added benchmark.

Closes #24637 from viirya/SPARK-27707.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-18 23:32:07 -07:00
Huaxin Gao cd676e9f5e [SPARK-28277][SQL][PYTHON][TESTS] Convert and port 'except.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from ```except.sql``` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).
<details><summary>Diff comparing to 'except.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out
index c9b712d4d2..27ca7ea226 100644
--- a/sql/core/src/test/resources/sql-tests/results/except.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out
 -30,16 +30,16  struct<>

 -- !query 2
-SELECT * FROM t1 EXCEPT SELECT * FROM t2
+SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2
 -- !query 2 schema
-struct<k:string,v:int>
+struct<CAST(udf(cast(k as string)) AS STRING):string,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 2 output
 three  3
 two    2

 -- !query 3
-SELECT * FROM t1 EXCEPT SELECT * FROM t1 where v <> 1 and v <> 2
+SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2)
 -- !query 3 schema
 struct<k:string,v:int>
 -- !query 3 output
 -49,7 +49,7  two   2

 -- !query 4
-SELECT * FROM t1 where v <> 1 and v <> 22 EXCEPT SELECT * FROM t1 where v <> 2 and v >= 3
+SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3)
 -- !query 4 schema
 struct<k:string,v:int>
 -- !query 4 output
 -59,7 +59,7  two   2
 -- !query 5
 SELECT t1.* FROM t1, t2 where t1.k = t2.k
 EXCEPT
-SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != 'one'
+SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one')
 -- !query 5 schema
 struct<k:string,v:int>
 -- !query 5 output
 -68,7 +68,7  one   NULL

 -- !query 6
-SELECT * FROM t2 where v >= 1 and v <> 22 EXCEPT SELECT * FROM t1
+SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1
 -- !query 6 schema
 struct<k:string,v:int>
 -- !query 6 output
 -77,9 +77,9  one   5

 -- !query 7
-SELECT (SELECT min(k) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1
+SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1
 MINUS
-SELECT (SELECT min(k) FROM t2) abs_min_t2 FROM t1 WHERE  t1.k = 'one'
+SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE  t1.k = udf('one')
 -- !query 7 schema
 struct<min_t2:string>
 -- !query 7 output
 -90,16 +90,17  two
 -- !query 8
 SELECT t1.k
 FROM   t1
-WHERE  t1.v <= (SELECT   max(t2.v)
+WHERE  t1.v <= (SELECT   udf(max(udf(t2.v)))
                 FROM     t2
-                WHERE    t2.k = t1.k)
+                WHERE    udf(t2.k) = udf(t1.k))
 MINUS
 SELECT t1.k
 FROM   t1
-WHERE  t1.v >= (SELECT   min(t2.v)
+WHERE  udf(t1.v) >= (SELECT   min(udf(t2.v))
                 FROM     t2
                 WHERE    t2.k = t1.k)
 -- !query 8 schema
-struct<k:string>
+struct<>
 -- !query 8 output
-two
+java.lang.UnsupportedOperationException
+Cannot evaluate expression: udf(cast(null as string))
```

</p>
</details>

## How was this patch tested?
Tested as guided in [SPARK-27921.](https://issues.apache.org/jira/browse/SPARK-27921)

Closes #25101 from huaxingao/spark-28277.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-19 13:44:26 +09:00
Huaxin Gao 20578e81a7 [SPARK-28285][SQL][PYTHON][TESTS] Convert and port 'outer-join.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from ```outer-join.sql``` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).
<details><summary>Diff comparing to 'outer-join.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out
index 5db3bae5d0..819f786070 100644
--- a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out
 -24,17 +24,17  struct<>

 -- !query 2
 SELECT
-  (SUM(COALESCE(t1.int_col1, t2.int_col0))),
-     ((COALESCE(t1.int_col1, t2.int_col0)) * 2)
+  (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))),
+     (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2)
 FROM t1
 RIGHT JOIN t2
-  ON (t2.int_col0) = (t1.int_col1)
-GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)),
+  ON udf(t2.int_col0) = udf(t1.int_col1)
+GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf(-449)))),
          COALESCE(t1.int_col1, t2.int_col0)
-HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0)))
-            > ((COALESCE(t1.int_col1, t2.int_col0)) * 2)
+HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0)))))
+            > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2)
 -- !query 2 schema
-struct<sum(coalesce(int_col1, int_col0)):bigint,(coalesce(int_col1, int_col0) * 2):int>
+struct<CAST(udf(cast(sum(cast(cast(udf(cast(coalesce(int_col1, int_col0) as string)) as int) as bigint)) as string)) AS BIGINT):bigint,(CAST(udf(cast(coalesce(int_col1, int_col0) as string)) AS INT) * 2):int>
 -- !query 2 output
 -367   -734
 -507   -1014
 -70,10 +70,10  spark.sql.crossJoin.enabled true
 SELECT *
 FROM (
 SELECT
-    COALESCE(t2.int_col1, t1.int_col1) AS int_col
+    udf(COALESCE(udf(t2.int_col1), udf(t1.int_col1))) AS int_col
     FROM t1
     LEFT JOIN t2 ON false
-) t where (t.int_col) is not null
+) t where (udf(t.int_col)) is not null
 -- !query 6 schema
 struct<int_col:int>
 -- !query 6 output
```

</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25103 from huaxingao/spark-28285.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-19 12:16:41 +09:00
Vinod KC d2598fee3b [SPARK-28287][SQL][PYTHON][TESTS] Convert and port 'udaf.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from 'udaf.sql' to test UDFs

<details><summary>Diff comparing to 'udaf.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out
index f4455bb717..e1747f4667 100644
--- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out
 -3,6 +3,8

 -- !query 0
+-- This test file was converted from udaf.sql.
+
 CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES
 (1), (2), (3), (4)
 as t1(int_col1)
 -21,15 +23,15  struct<>

 -- !query 2
-SELECT default.myDoubleAvg(int_col1) as my_avg from t1
+SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1
 -- !query 2 schema
-struct<my_avg:double>
+struct<my_avg:double,my_avg2:double,my_avg3:double>
 -- !query 2 output
-102.5
+102.5  102.5   102.5

 -- !query 3
-SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1
+SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1
 -- !query 3 schema
 struct<>
 -- !query 3 output
 -46,12 +48,12  struct<>

 -- !query 5
-SELECT default.udaf1(int_col1) as udaf1 from t1
+SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1
 -- !query 5 schema
 struct<>
 -- !query 5 output
 org.apache.spark.sql.AnalysisException
-Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 7
+Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 94

 -- !query 6
```

</p>
</details>

## How was this patch tested?

Tested as guided in SPARK-27921.

Closes #25194 from vinodkc/br_Fix_SPARK-27921_3.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-19 10:48:13 +09:00
Maxim Gekk 54e058dff2 [SPARK-28416][SQL] Use java.time API in timestampAddInterval
## What changes were proposed in this pull request?

The `DateTimeUtils.timestampAddInterval` method was rewritten by using Java 8 time API. To add months and microseconds, I used the `plusMonths()` and `plus()` methods of `ZonedDateTime`. Also the signature of `timestampAddInterval()` was changed to accept an `ZoneId` instance instead of `TimeZone`. Using `ZoneId` allows to avoid the conversion `TimeZone` -> `ZoneId` on every invoke of `timestampAddInterval()`.

## How was this patch tested?

By existing test suites `DateExpressionsSuite`, `TypeCoercionSuite` and `CollectionExpressionsSuite`.

Closes #25173 from MaxGekk/timestamp-add-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-18 19:17:23 -04:00
Yuming Wang 0c21404f7c [SPARK-28312][SQL][TEST] Port numeric.sql
## What changes were proposed in this pull request?

This PR is to port numeric.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/numeric.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/numeric.out

When porting the test cases, found four PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28315](https://issues.apache.org/jira/browse/SPARK-28315): Decimal can not accept `NaN` as input
[SPARK-28317](https://issues.apache.org/jira/browse/SPARK-28317): Built-in Mathematical Functions: SCALE
[SPARK-28318](https://issues.apache.org/jira/browse/SPARK-28318): Decimal can only support precision up to 38
[SPARK-28322](https://issues.apache.org/jira/browse/SPARK-28322): DIV support decimal type

Also, found four inconsistent behavior:
[SPARK-28316](https://issues.apache.org/jira/browse/SPARK-28316): Decimal precision issue
[SPARK-28324](https://issues.apache.org/jira/browse/SPARK-28324): The LOG function using 10 as the base, but Spark using E
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Spark SQL insert bad inputs to NULL
[SPARK-28007](https://issues.apache.org/jira/browse/SPARK-28007):  Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres

## How was this patch tested?

N/A

Closes #25092 from wangyum/SPARK-28312.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-18 13:49:51 -07:00
Yuming Wang 2cf0491a97 [SPARK-28388][SQL][TEST] Port select_implicit.sql
## What changes were proposed in this pull request?

This PR is to port numeric.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_implicit.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/select_implicit.out

When porting the test cases, found one PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28329](https://issues.apache.org/jira/browse/SPARK-28329): SELECT INTO syntax

## How was this patch tested?

N/A

Closes #25152 from wangyum/SPARK-28388.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-18 08:58:27 -07:00
Yuming Wang 8acc22ca64 [SPARK-28138][SQL][TEST] Port timestamp.sql
## What changes were proposed in this pull request?

This PR is to port timestamp.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/timestamp.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/timestamp.out

When porting the test cases, found five PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28141](https://issues.apache.org/jira/browse/SPARK-28141): Timestamp type can not accept special values
[SPARK-28259](https://issues.apache.org/jira/browse/SPARK-28259): Date/Time Output Styles and Date Order Conventions
[SPARK-28425](https://issues.apache.org/jira/browse/SPARK-28425): Add more Date/Time Operators
[SPARK-28420](https://issues.apache.org/jira/browse/SPARK-28420): Date/Time Functions: date_part
[SPARK-28137](https://issues.apache.org/jira/browse/SPARK-28137): Data Type Formatting Functions
[SPARK-28432](https://issues.apache.org/jira/browse/SPARK-28432): Date/Time Functions: make_date/make_timestamp

Also, found one inconsistent behavior:
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Spark SQL insert bad inputs to NULL

## How was this patch tested?

N/A

Closes #25181 from wangyum/SPARK-28138.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-18 08:50:31 -07:00
chitralverma 4b865104b3 [SPARK-28286][SQL][PYTHON][TESTS] Convert and port 'pivot.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from pivot.sql to test UDFs following the combination guide in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'pivot.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out
index 9a8f783da4..cb9e4d736c 100644
--- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out
 -1,5 +1,5
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 32
+-- Number of queries: 30

 -- !query 0
 -40,14 +40,14  struct<>

 -- !query 3
 SELECT * FROM (
-  SELECT year, course, earnings FROM courseSales
+  SELECT udf(year), course, earnings FROM courseSales
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 3 schema
-struct<year:int,dotNET:bigint,Java:bigint>
+struct<CAST(udf(cast(year as string)) AS INT):int,dotNET:bigint,Java:bigint>
 -- !query 3 output
 2012   15000   20000
 2013   48000   30000
 -56,7 +56,7  struct<year:int,dotNET:bigint,Java:bigint>
 -- !query 4
 SELECT * FROM courseSales
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR year IN (2012, 2013)
 )
 -- !query 4 schema
 -71,11 +71,11  SELECT * FROM (
   SELECT year, course, earnings FROM courseSales
 )
 PIVOT (
-  sum(earnings), avg(earnings)
+  udf(sum(earnings)), udf(avg(earnings))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 5 schema
-struct<year:int,dotNET_sum(CAST(earnings AS BIGINT)):bigint,dotNET_avg(CAST(earnings AS BIGINT)):double,Java_sum(CAST(earnings AS BIGINT)):bigint,Java_avg(CAST(earnings AS BIGINT)):double>
+struct<year:int,dotNET_CAST(udf(cast(sum(cast(earnings as bigint)) as string)) AS BIGINT):bigint,dotNET_CAST(udf(cast(avg(cast(earnings as bigint)) as string)) AS DOUBLE):double,Java_CAST(udf(cast(sum(cast(earnings as bigint)) as string)) AS BIGINT):bigint,Java_CAST(udf(cast(avg(cast(earnings as bigint)) as string)) AS DOUBLE):double>
 -- !query 5 output
 2012   15000   7500.0  20000   20000.0
 2013   48000   48000.0 30000   30000.0
 -83,10 +83,10  struct<year:int,dotNET_sum(CAST(earnings AS BIGINT)):bigint,dotNET_avg(CAST(earn

 -- !query 6
 SELECT * FROM (
-  SELECT course, earnings FROM courseSales
+  SELECT udf(course) as course, earnings FROM courseSales
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 6 schema
 -100,23 +100,23  SELECT * FROM (
   SELECT year, course, earnings FROM courseSales
 )
 PIVOT (
-  sum(earnings), min(year)
+  udf(sum(udf(earnings))), udf(min(year))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 7 schema
-struct<dotNET_sum(CAST(earnings AS BIGINT)):bigint,dotNET_min(year):int,Java_sum(CAST(earnings AS BIGINT)):bigint,Java_min(year):int>
+struct<dotNET_CAST(udf(cast(sum(cast(cast(udf(cast(earnings as string)) as int) as bigint)) as string)) AS BIGINT):bigint,dotNET_CAST(udf(cast(min(year) as string)) AS INT):int,Java_CAST(udf(cast(sum(cast(cast(udf(cast(earnings as string)) as int) as bigint)) as string)) AS BIGINT):bigint,Java_CAST(udf(cast(min(year) as string)) AS INT):int>
 -- !query 7 output
 63000  2012    50000   2012

 -- !query 8
 SELECT * FROM (
-  SELECT course, year, earnings, s
+  SELECT course, year, earnings, udf(s) as s
   FROM courseSales
   JOIN years ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR s IN (1, 2)
 )
 -- !query 8 schema
 -135,11 +135,11  SELECT * FROM (
   JOIN years ON year = y
 )
 PIVOT (
-  sum(earnings), min(s)
+  udf(sum(earnings)), udf(min(s))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 9 schema
-struct<year:int,dotNET_sum(CAST(earnings AS BIGINT)):bigint,dotNET_min(s):int,Java_sum(CAST(earnings AS BIGINT)):bigint,Java_min(s):int>
+struct<year:int,dotNET_CAST(udf(cast(sum(cast(earnings as bigint)) as string)) AS BIGINT):bigint,dotNET_CAST(udf(cast(min(s) as string)) AS INT):int,Java_CAST(udf(cast(sum(cast(earnings as bigint)) as string)) AS BIGINT):bigint,Java_CAST(udf(cast(min(s) as string)) AS INT):int>
 -- !query 9 output
 2012   15000   1       20000   1
 2013   48000   2       30000   2
 -152,7 +152,7  SELECT * FROM (
   JOIN years ON year = y
 )
 PIVOT (
-  sum(earnings * s)
+  udf(sum(earnings * s))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 10 schema
 -167,7 +167,7  SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM (
   SELECT year y, course c, earnings e FROM courseSales
 )
 PIVOT (
-  sum(e) s, avg(e) a
+  udf(sum(e)) s, udf(avg(e)) a
   FOR y IN (2012, 2013)
 )
 -- !query 11 schema
 -182,7 +182,7  SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM (
   SELECT year y, course c, earnings e FROM courseSales
 )
 PIVOT (
-  sum(e) s, avg(e) a
+  udf(sum(e)) s, udf(avg(e)) a
   FOR y IN (2012 as firstYear, 2013 secondYear)
 )
 -- !query 12 schema
 -195,7 +195,7  struct<firstYear_s:bigint,secondYear_s:bigint,firstYear_a:double,secondYear_a:do
 -- !query 13
 SELECT * FROM courseSales
 PIVOT (
-  abs(earnings)
+  udf(abs(earnings))
   FOR year IN (2012, 2013)
 )
 -- !query 13 schema
 -210,7 +210,7  SELECT * FROM (
   SELECT year, course, earnings FROM courseSales
 )
 PIVOT (
-  sum(earnings), year
+  udf(sum(earnings)), year
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 14 schema
 -225,7 +225,7  SELECT * FROM (
   SELECT course, earnings FROM courseSales
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR year IN (2012, 2013)
 )
 -- !query 15 schema
 -240,11 +240,11  SELECT * FROM (
   SELECT year, course, earnings FROM courseSales
 )
 PIVOT (
-  ceil(sum(earnings)), avg(earnings) + 1 as a1
+  udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 16 schema
-struct<year:int,dotNET_CEIL(sum(CAST(earnings AS BIGINT))):bigint,dotNET_a1:double,Java_CEIL(sum(CAST(earnings AS BIGINT))):bigint,Java_a1:double>
+struct<year:int,dotNET_CAST(udf(cast(CEIL(cast(udf(cast(sum(cast(earnings as bigint)) as string)) as bigint)) as string)) AS BIGINT):bigint,dotNET_a1:double,Java_CAST(udf(cast(CEIL(cast(udf(cast(sum(cast(earnings as bigint)) as string)) as bigint)) as string)) AS BIGINT):bigint,Java_a1:double>
 -- !query 16 output
 2012   15000   7501.0  20000   20001.0
 2013   48000   48001.0 30000   30001.0
 -255,7 +255,7  SELECT * FROM (
   SELECT year, course, earnings FROM courseSales
 )
 PIVOT (
-  sum(avg(earnings))
+  sum(udf(avg(earnings)))
   FOR course IN ('dotNET', 'Java')
 )
 -- !query 17 schema
 -272,7 +272,7  SELECT * FROM (
   JOIN years ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR (course, year) IN (('dotNET', 2012), ('Java', 2013))
 )
 -- !query 18 schema
 -289,7 +289,7  SELECT * FROM (
   JOIN years ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2)
 )
 -- !query 19 schema
 -306,7 +306,7  SELECT * FROM (
   JOIN years ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR (course, year) IN ('dotNET', 'Java')
 )
 -- !query 20 schema
 -319,7 +319,7  Invalid pivot value 'dotNET': value data type string does not match pivot column
 -- !query 21
 SELECT * FROM courseSales
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR year IN (s, 2013)
 )
 -- !query 21 schema
 -332,7 +332,7  cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earni
 -- !query 22
 SELECT * FROM courseSales
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR year IN (course, 2013)
 )
 -- !query 22 schema
 -343,151 +343,118  Literal expressions required for pivot values, found 'course#x';

 -- !query 23
-SELECT * FROM (
-  SELECT course, year, a
-  FROM courseSales
-  JOIN yearsWithComplexTypes ON year = y
-)
-PIVOT (
-  min(a)
-  FOR course IN ('dotNET', 'Java')
-)
--- !query 23 schema
-struct<year:int,dotNET:array<int>,Java:array<int>>
--- !query 23 output
-2012   [1,1]   [1,1]
-2013   [2,2]   [2,2]
-
-
--- !query 24
-SELECT * FROM (
-  SELECT course, year, y, a
-  FROM courseSales
-  JOIN yearsWithComplexTypes ON year = y
-)
-PIVOT (
-  max(a)
-  FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java'))
-)
--- !query 24 schema
-struct<year:int,[2012, dotNET]:array<int>,[2013, Java]:array<int>>
--- !query 24 output
-2012   [1,1]   NULL
-2013   NULL    [2,2]
-
-
--- !query 25
 SELECT * FROM (
   SELECT earnings, year, a
   FROM courseSales
   JOIN yearsWithComplexTypes ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR a IN (array(1, 1), array(2, 2))
 )
--- !query 25 schema
+-- !query 23 schema
 struct<year:int,[1, 1]:bigint,[2, 2]:bigint>
--- !query 25 output
+-- !query 23 output
 2012   35000   NULL
 2013   NULL    78000

--- !query 26
+-- !query 24
 SELECT * FROM (
-  SELECT course, earnings, year, a
+  SELECT course, earnings, udf(year) as year, a
   FROM courseSales
   JOIN yearsWithComplexTypes ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2)))
 )
--- !query 26 schema
+-- !query 24 schema
 struct<year:int,[dotNET, [1, 1]]:bigint,[Java, [2, 2]]:bigint>
--- !query 26 output
+-- !query 24 output
 2012   15000   NULL
 2013   NULL    30000

--- !query 27
+-- !query 25
 SELECT * FROM (
   SELECT earnings, year, s
   FROM courseSales
   JOIN yearsWithComplexTypes ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR s IN ((1, 'a'), (2, 'b'))
 )
--- !query 27 schema
+-- !query 25 schema
 struct<year:int,[1, a]:bigint,[2, b]:bigint>
--- !query 27 output
+-- !query 25 output
 2012   35000   NULL
 2013   NULL    78000

--- !query 28
+-- !query 26
 SELECT * FROM (
   SELECT course, earnings, year, s
   FROM courseSales
   JOIN yearsWithComplexTypes ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b')))
 )
--- !query 28 schema
+-- !query 26 schema
 struct<year:int,[dotNET, [1, a]]:bigint,[Java, [2, b]]:bigint>
--- !query 28 output
+-- !query 26 output
 2012   15000   NULL
 2013   NULL    30000

--- !query 29
+-- !query 27
 SELECT * FROM (
   SELECT earnings, year, m
   FROM courseSales
   JOIN yearsWithComplexTypes ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR m IN (map('1', 1), map('2', 2))
 )
--- !query 29 schema
+-- !query 27 schema
 struct<>
--- !query 29 output
+-- !query 27 output
 org.apache.spark.sql.AnalysisException
 Invalid pivot column 'm#x'. Pivot columns must be comparable.;

--- !query 30
+-- !query 28
 SELECT * FROM (
   SELECT course, earnings, year, m
   FROM courseSales
   JOIN yearsWithComplexTypes ON year = y
 )
 PIVOT (
-  sum(earnings)
+  udf(sum(earnings))
   FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2)))
 )
--- !query 30 schema
+-- !query 28 schema
 struct<>
--- !query 30 output
+-- !query 28 output
 org.apache.spark.sql.AnalysisException
 Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.;

--- !query 31
+-- !query 29
 SELECT * FROM (
-  SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w
+  SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y,
+  udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w
   FROM courseSales
 )
 PIVOT (
-  sum(Earnings)
+  udf(sum(Earnings))
   FOR Course IN ('dotNET', 'Java')
 )
--- !query 31 schema
+-- !query 29 schema
 struct<a:string,z:string,b:string,y:string,c:string,x:string,d:string,w:string,dotNET:bigint,Java:bigint>
--- !query 31 output
+-- !query 29 output
 a      z       b       y       c       x       d       w       63000   50000

```

</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25122 from chitralverma/SPARK-28286.

Authored-by: chitralverma <chitralverma@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-18 22:19:14 +09:00
Terry Kim eaaf1aa2ac [SPARK-28278][SQL][PYTHON][TESTS] Convert and port 'except-all.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `except-all.sql` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'except-all.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out
index 01091a2f75..b7bfad0e53 100644
--- a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out
 -49,11 +49,11  struct<>

 -- !query 4
-SELECT * FROM tab1
+SELECT udf(c1) FROM tab1
 EXCEPT ALL
-SELECT * FROM tab2
+SELECT udf(c1) FROM tab2
 -- !query 4 schema
-struct<c1:int>
+struct<CAST(udf(cast(c1 as string)) AS INT):int>
 -- !query 4 output
 0
 2
 -62,11 +62,11  NULL

 -- !query 5
-SELECT * FROM tab1
+SELECT udf(c1) FROM tab1
 MINUS ALL
-SELECT * FROM tab2
+SELECT udf(c1) FROM tab2
 -- !query 5 schema
-struct<c1:int>
+struct<CAST(udf(cast(c1 as string)) AS INT):int>
 -- !query 5 output
 0
 2
 -75,11 +75,11  NULL

 -- !query 6
-SELECT * FROM tab1
+SELECT udf(c1) FROM tab1
 EXCEPT ALL
-SELECT * FROM tab2 WHERE c1 IS NOT NULL
+SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL
 -- !query 6 schema
-struct<c1:int>
+struct<CAST(udf(cast(c1 as string)) AS INT):int>
 -- !query 6 output
 0
 2
 -89,21 +89,21  NULL

 -- !query 7
-SELECT * FROM tab1 WHERE c1 > 5
+SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5
 EXCEPT ALL
-SELECT * FROM tab2
+SELECT udf(c1) FROM tab2
 -- !query 7 schema
-struct<c1:int>
+struct<CAST(udf(cast(c1 as string)) AS INT):int>
 -- !query 7 output

 -- !query 8
-SELECT * FROM tab1
+SELECT udf(c1) FROM tab1
 EXCEPT ALL
-SELECT * FROM tab2 WHERE c1 > 6
+SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6))
 -- !query 8 schema
-struct<c1:int>
+struct<CAST(udf(cast(c1 as string)) AS INT):int>
 -- !query 8 output
 0
 1
 -117,11 +117,11  NULL

 -- !query 9
-SELECT * FROM tab1
+SELECT udf(c1) FROM tab1
 EXCEPT ALL
-SELECT CAST(1 AS BIGINT)
+SELECT CAST(udf(1) AS BIGINT)
 -- !query 9 schema
-struct<c1:bigint>
+struct<CAST(udf(cast(c1 as string)) AS INT):bigint>
 -- !query 9 output
 0
 2
 -134,7 +134,7  NULL

 -- !query 10
-SELECT * FROM tab1
+SELECT udf(c1) FROM tab1
 EXCEPT ALL
 SELECT array(1)
 -- !query 10 schema
 -145,62 +145,62  ExceptAll can only be performed on tables with the compatible column types. arra

 -- !query 11
-SELECT * FROM tab3
+SELECT udf(k), v FROM tab3
 EXCEPT ALL
-SELECT * FROM tab4
+SELECT k, udf(v) FROM tab4
 -- !query 11 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 11 output
 1	2
 1	3

 -- !query 12
-SELECT * FROM tab4
+SELECT k, udf(v) FROM tab4
 EXCEPT ALL
-SELECT * FROM tab3
+SELECT udf(k), v FROM tab3
 -- !query 12 schema
-struct<k:int,v:int>
+struct<k:int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 12 output
 2	2
 2	20

 -- !query 13
-SELECT * FROM tab4
+SELECT udf(k), udf(v) FROM tab4
 EXCEPT ALL
-SELECT * FROM tab3
+SELECT udf(k), udf(v) FROM tab3
 INTERSECT DISTINCT
-SELECT * FROM tab4
+SELECT udf(k), udf(v) FROM tab4
 -- !query 13 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 13 output
 2	2
 2	20

 -- !query 14
-SELECT * FROM tab4
+SELECT udf(k), v FROM tab4
 EXCEPT ALL
-SELECT * FROM tab3
+SELECT k, udf(v) FROM tab3
 EXCEPT DISTINCT
-SELECT * FROM tab4
+SELECT udf(k), udf(v) FROM tab4
 -- !query 14 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 14 output

 -- !query 15
-SELECT * FROM tab3
+SELECT k, udf(v) FROM tab3
 EXCEPT ALL
-SELECT * FROM tab4
+SELECT udf(k), udf(v) FROM tab4
 UNION ALL
-SELECT * FROM tab3
+SELECT udf(k), v FROM tab3
 EXCEPT DISTINCT
-SELECT * FROM tab4
+SELECT k, udf(v) FROM tab4
 -- !query 15 schema
-struct<k:int,v:int>
+struct<k:int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 15 output
 1	3

 -217,83 +217,83  ExceptAll can only be performed on tables with the same number of columns, but t

 -- !query 17
-SELECT * FROM tab3
+SELECT udf(k), udf(v) FROM tab3
 EXCEPT ALL
-SELECT * FROM tab4
+SELECT udf(k), udf(v) FROM tab4
 UNION
-SELECT * FROM tab3
+SELECT udf(k), udf(v) FROM tab3
 EXCEPT DISTINCT
-SELECT * FROM tab4
+SELECT udf(k), udf(v) FROM tab4
 -- !query 17 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 17 output
 1	3

 -- !query 18
-SELECT * FROM tab3
+SELECT udf(k), udf(v) FROM tab3
 MINUS ALL
-SELECT * FROM tab4
+SELECT k, udf(v) FROM tab4
 UNION
-SELECT * FROM tab3
+SELECT udf(k), udf(v) FROM tab3
 MINUS DISTINCT
-SELECT * FROM tab4
+SELECT k, udf(v) FROM tab4
 -- !query 18 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 18 output
 1	3

 -- !query 19
-SELECT * FROM tab3
+SELECT k, udf(v) FROM tab3
 EXCEPT ALL
-SELECT * FROM tab4
+SELECT udf(k), v FROM tab4
 EXCEPT DISTINCT
-SELECT * FROM tab3
+SELECT k, udf(v) FROM tab3
 EXCEPT DISTINCT
-SELECT * FROM tab4
+SELECT udf(k), v FROM tab4
 -- !query 19 schema
-struct<k:int,v:int>
+struct<k:int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 19 output

 -- !query 20
 SELECT *
-FROM   (SELECT tab3.k,
-               tab4.v
+FROM   (SELECT tab3.k,
+               udf(tab4.v)
         FROM   tab3
                JOIN tab4
-                 ON tab3.k = tab4.k)
+                 ON udf(tab3.k) = tab4.k)
 EXCEPT ALL
 SELECT *
-FROM   (SELECT tab3.k,
-               tab4.v
+FROM   (SELECT udf(tab3.k),
+               tab4.v
         FROM   tab3
                JOIN tab4
-                 ON tab3.k = tab4.k)
+                 ON tab3.k = udf(tab4.k))
 -- !query 20 schema
-struct<k:int,v:int>
+struct<k:int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 20 output

 -- !query 21
 SELECT *
-FROM   (SELECT tab3.k,
-               tab4.v
+FROM   (SELECT udf(udf(tab3.k)),
+               udf(tab4.v)
         FROM   tab3
                JOIN tab4
-                 ON tab3.k = tab4.k)
+                 ON udf(udf(tab3.k)) = udf(tab4.k))
 EXCEPT ALL
 SELECT *
-FROM   (SELECT tab4.v AS k,
-               tab3.k AS v
+FROM   (SELECT udf(tab4.v) AS k,
+               udf(udf(tab3.k)) AS v
         FROM   tab3
                JOIN tab4
-                 ON tab3.k = tab4.k)
+                 ON udf(tab3.k) = udf(tab4.k))
 -- !query 21 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 21 output
 1	2
 1	2
 -305,11 +305,11  struct<k:int,v:int>

 -- !query 22
-SELECT v FROM tab3 GROUP BY v
+SELECT udf(v) FROM tab3 GROUP BY v
 EXCEPT ALL
-SELECT k FROM tab4 GROUP BY k
+SELECT udf(k) FROM tab4 GROUP BY k
 -- !query 22 schema
-struct<v:int>
+struct<CAST(udf(cast(v as string)) AS INT):int>
 -- !query 22 output
 3

```
</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25090 from imback82/except-all.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-18 19:51:50 +09:00
Terry Kim 62004f1c0f [SPARK-28283][SQL][PYTHON][TESTS] Convert and port 'intersect-all.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `intersect-all.sql` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'intersect-all.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out
index 63dd56ce46..0cb82be2da 100644
--- a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out
 -34,11 +34,11  struct<>

 -- !query 2
-SELECT * FROM tab1
+SELECT udf(k), v FROM tab1
 INTERSECT ALL
-SELECT * FROM tab2
+SELECT k, udf(v) FROM tab2
 -- !query 2 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 2 output
 1	2
 1	2
 -48,11 +48,11  NULL	NULL

 -- !query 3
-SELECT * FROM tab1
+SELECT k, udf(v) FROM tab1
 INTERSECT ALL
-SELECT * FROM tab1 WHERE k = 1
+SELECT udf(k), v FROM tab1 WHERE udf(k) = 1
 -- !query 3 schema
-struct<k:int,v:int>
+struct<k:int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 3 output
 1	2
 1	2
 -61,39 +61,39  struct<k:int,v:int>

 -- !query 4
-SELECT * FROM tab1 WHERE k > 2
+SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2)
 INTERSECT ALL
-SELECT * FROM tab2
+SELECT udf(k), udf(v) FROM tab2
 -- !query 4 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 4 output

 -- !query 5
-SELECT * FROM tab1
+SELECT udf(k), v FROM tab1
 INTERSECT ALL
-SELECT * FROM tab2 WHERE k > 3
+SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3
 -- !query 5 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 5 output

 -- !query 6
-SELECT * FROM tab1
+SELECT udf(k), v FROM tab1
 INTERSECT ALL
-SELECT CAST(1 AS BIGINT), CAST(2 AS BIGINT)
+SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT)
 -- !query 6 schema
-struct<k:bigint,v:bigint>
+struct<CAST(udf(cast(k as string)) AS INT):bigint,v:bigint>
 -- !query 6 output
 1	2

 -- !query 7
-SELECT * FROM tab1
+SELECT k, udf(v) FROM tab1
 INTERSECT ALL
-SELECT array(1), 2
+SELECT array(1), udf(2)
 -- !query 7 schema
 struct<>
 -- !query 7 output
 -102,9 +102,9  IntersectAll can only be performed on tables with the compatible column types. a

 -- !query 8
-SELECT k FROM tab1
+SELECT udf(k) FROM tab1
 INTERSECT ALL
-SELECT k, v FROM tab2
+SELECT udf(k), udf(v) FROM tab2
 -- !query 8 schema
 struct<>
 -- !query 8 output
 -113,13 +113,13  IntersectAll can only be performed on tables with the same number of columns, bu

 -- !query 9
-SELECT * FROM tab2
+SELECT udf(k), v FROM tab2
 INTERSECT ALL
-SELECT * FROM tab1
+SELECT k, udf(v) FROM tab1
 INTERSECT ALL
-SELECT * FROM tab2
+SELECT udf(k), udf(v) FROM tab2
 -- !query 9 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 9 output
 1	2
 1	2
 -129,15 +129,15  NULL	NULL

 -- !query 10
-SELECT * FROM tab1
+SELECT udf(k), v FROM tab1
 EXCEPT
-SELECT * FROM tab2
+SELECT k, udf(v) FROM tab2
 UNION ALL
-SELECT * FROM tab1
+SELECT k, udf(udf(v)) FROM tab1
 INTERSECT ALL
-SELECT * FROM tab2
+SELECT udf(k), v FROM tab2
 -- !query 10 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 10 output
 1	2
 1	2
 -148,15 +148,15  NULL	NULL

 -- !query 11
-SELECT * FROM tab1
+SELECT udf(k), udf(v) FROM tab1
 EXCEPT
-SELECT * FROM tab2
+SELECT udf(k), v FROM tab2
 EXCEPT
-SELECT * FROM tab1
+SELECT k, udf(v) FROM tab1
 INTERSECT ALL
-SELECT * FROM tab2
+SELECT udf(k), udf(udf(v)) FROM tab2
 -- !query 11 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 11 output
 1	3

 -165,38 +165,38  struct<k:int,v:int>
 (
   (
     (
-      SELECT * FROM tab1
+      SELECT udf(k), v FROM tab1
       EXCEPT
-      SELECT * FROM tab2
+      SELECT k, udf(v) FROM tab2
     )
     EXCEPT
-    SELECT * FROM tab1
+    SELECT udf(k), udf(v) FROM tab1
   )
   INTERSECT ALL
-  SELECT * FROM tab2
+  SELECT udf(k), udf(v) FROM tab2
 )
 -- !query 12 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 12 output

 -- !query 13
 SELECT *
-FROM   (SELECT tab1.k,
-               tab2.v
+FROM   (SELECT udf(tab1.k),
+               udf(tab2.v)
         FROM   tab1
                JOIN tab2
-                 ON tab1.k = tab2.k)
+                 ON udf(udf(tab1.k)) = tab2.k)
 INTERSECT ALL
 SELECT *
-FROM   (SELECT tab1.k,
-               tab2.v
+FROM   (SELECT udf(tab1.k),
+               udf(tab2.v)
         FROM   tab1
                JOIN tab2
-                 ON tab1.k = tab2.k)
+                 ON udf(tab1.k) = udf(udf(tab2.k)))
 -- !query 13 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 13 output
 1	2
 1	2
 -211,30 +211,30  struct<k:int,v:int>

 -- !query 14
 SELECT *
-FROM   (SELECT tab1.k,
-               tab2.v
+FROM   (SELECT udf(tab1.k),
+               udf(tab2.v)
         FROM   tab1
                JOIN tab2
-                 ON tab1.k = tab2.k)
+                 ON udf(tab1.k) = udf(tab2.k))
 INTERSECT ALL
 SELECT *
-FROM   (SELECT tab2.v AS k,
-               tab1.k AS v
+FROM   (SELECT udf(tab2.v) AS k,
+               udf(tab1.k) AS v
         FROM   tab1
                JOIN tab2
-                 ON tab1.k = tab2.k)
+                 ON tab1.k = udf(tab2.k))
 -- !query 14 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 14 output

 -- !query 15
-SELECT v FROM tab1 GROUP BY v
+SELECT udf(v) FROM tab1 GROUP BY v
 INTERSECT ALL
-SELECT k FROM tab2 GROUP BY k
+SELECT udf(udf(k)) FROM tab2 GROUP BY k
 -- !query 15 schema
-struct<v:int>
+struct<CAST(udf(cast(v as string)) AS INT):int>
 -- !query 15 output
 2
 3
 -250,15 +250,15  spark.sql.legacy.setopsPrecedence.enabled	true

 -- !query 17
-SELECT * FROM tab1
+SELECT udf(k), v FROM tab1
 EXCEPT
-SELECT * FROM tab2
+SELECT k, udf(v) FROM tab2
 UNION ALL
-SELECT * FROM tab1
+SELECT udf(k), udf(v) FROM tab1
 INTERSECT ALL
-SELECT * FROM tab2
+SELECT udf(udf(k)), udf(v) FROM tab2
 -- !query 17 schema
-struct<k:int,v:int>
+struct<CAST(udf(cast(k as string)) AS INT):int,v:int>
 -- !query 17 output
 1	2
 1	2
 -268,15 +268,15  NULL	NULL

 -- !query 18
-SELECT * FROM tab1
+SELECT k, udf(v) FROM tab1
 EXCEPT
-SELECT * FROM tab2
+SELECT udf(k), v FROM tab2
 UNION ALL
-SELECT * FROM tab1
+SELECT udf(k), udf(v) FROM tab1
 INTERSECT
-SELECT * FROM tab2
+SELECT udf(k), udf(udf(v)) FROM tab2
 -- !query 18 schema
-struct<k:int,v:int>
+struct<k:int,CAST(udf(cast(v as string)) AS INT):int>
 -- !query 18 output
 1	2
 2	3

```
</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25119 from imback82/intersect-all-sql.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-18 19:49:57 +09:00
Liang-Chi Hsieh 4645ffb08a [SPARK-28276][SQL][PYTHON][TEST] Convert and port 'cross-join.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `cross-join.sql'` to test UDFs.

<details><summary>Diff comparing to 'cross-join.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out
index 3833c42bdf..11c1e01d54 100644
--- a/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out
 -43,7 +43,7  two   2       two     22

 -- !query 3
-SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k
+SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k)
 -- !query 3 schema
 struct<k:string,v1:int,k:string,v2:int>
 -- !query 3 output
 -53,7 +53,7  two   2       two     22

 -- !query 4
-SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k)
+SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k))
 -- !query 4 schema
 struct<k:string,v1:int,k:string,v2:int>
 -- !query 4 output
 -63,7 +63,7  two   2       two     22

 -- !query 5
-SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22
+SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22"
 -- !query 5 schema
 struct<k:string,v1:int,k:string,v2:int>
 -- !query 5 output
 -71,12 +71,12  one 1       two     22

-- !query 6
-SELECT a.key, b.key FROM
-(SELECT k key FROM nt1 WHERE v1 < 2) a
+SELECT udf(a.key), udf(b.key) FROM
+(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a
 CROSS JOIN
-(SELECT k key FROM nt2 WHERE v2 = 22) b
+(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b
 -- !query 6 schema
-struct<key:string,key:string>
+struct<udf(key):string,udf(key):string>
 -- !query 6 output
 one    two

 -114,23 +114,29  struct<>

-- !query 11
-select * from ((A join B on (a = b)) cross join C) join D on (a = d)
+select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d))
 -- !query 11 schema
-struct<a:string,va:int,b:string,vb:int,c:string,vc:int,d:string,vd:int>
+struct<>
 -- !query 11 output
-one    1       one     1       one     1       one     1
-one    1       one     1       three   3       one     1
-one    1       one     1       two     2       one     1
-three  3       three   3       one     1       three   3
-three  3       three   3       three   3       three   3
-three  3       three   3       two     2       three   3
-two    2       two     2       one     1       two     2
-two    2       two     2       three   3       two     2
-two    2       two     2       two     2       two     2
+org.apache.spark.sql.AnalysisException
+Detected implicit cartesian product for INNER join between logical plans
+Filter (udf(a#x) = udf(b#x))
++- Join Inner
+   :- Project [k#x AS a#x, v1#x AS va#x]
+   :  +- LocalRelation [k#x, v1#x]
+   +- Project [k#x AS b#x, v1#x AS vb#x]
+      +- LocalRelation [k#x, v1#x]
+and
+Project [k#x AS d#x, v1#x AS vd#x]
++- LocalRelation [k#x, v1#x]
+Join condition is missing or trivial.
+Either: use the CROSS JOIN syntax to allow cartesian products between these
+relations, or: enable implicit cartesian products by setting the configuration
+variable spark.sql.crossJoin.enabled=true;

 -- !query 12
-SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k)
+SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k))
 -- !query 12 schema
 struct<k:string,v1:int,k:string,v2:int>
 -- !query 12 output
```

</p>
</details>

## How was this patch tested?

Added test.

Closes #25168 from viirya/SPARK-28276.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-18 19:46:00 +09:00
Seth Fitzsimmons eb5dc746c2 [SPARK-28097][SQL] Map ByteType to SMALLINT for PostgresDialect
## What changes were proposed in this pull request?

PostgreSQL doesn't have `TINYINT`, which would map directly, but `SMALLINT`s are sufficient for uni-directional translation.

A side-effect of this fix is that `AggregatedDialect` is now usable with multiple dialects targeting `jdbc:postgresql`, as `PostgresDialect.getJDBCType` no longer throws (for which reason backporting this fix would be lovely):

1217996f15/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala (L42)

`dialects.flatMap` currently throws on the first attempt to get a JDBC type preventing subsequent dialects in the chain from providing an alternative.

## How was this patch tested?

Unit tests.

Closes #24845 from mojodna/postgres-byte-type-mapping.

Authored-by: Seth Fitzsimmons <seth@mojodna.net>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-17 15:10:01 -07:00
HyukjinKwon 28774cd2ea [SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making UDFs (virtually) no-op
## What changes were proposed in this pull request?

Current UDFs available in `IntegratedUDFTestUtils` are not exactly no-op. It converts input column to strings and outputs to strings.

It causes some issues when we convert and port the tests at SPARK-27921. Integrated UDF test cases share one output file and it should outputs the same. However,

1. Special values are converted into strings differently:

    | Scala      | Python |
    | ---------- | ------ |
    | `null`     | `None` |
    | `Infinity` | `inf`  |
    | `-Infinity`| `-inf` |
    | `NaN`      | `nan`  |

2. Due to float limitation at Python (see https://docs.python.org/3/tutorial/floatingpoint.html), if float is passed into Python and sent back to JVM, the values are potentially not exactly correct. See https://github.com/apache/spark/pull/25128 and https://github.com/apache/spark/pull/25110

To work around this, this PR targets to change the current UDF to be wrapped by cast. So, Input column is casted into string, UDF returns strings as are, and then output column is casted back to the input column.

Roughly:

**Before:**

```
JVM (col1) -> (cast to string within Python) Python (string) -> (string) JVM
```

**After:**

```
JVM (cast col1 to string) -> (string) Python (string) -> (cast back to col1's type) JVM
```

In this way, UDF is virtually no-op although there might be some subtleties due to roundtrip in string cast. I believe this is good enough.

Python native functions and Scala native functions will take strings and output strings as are. So, there will be no potential test failures due to differences of conversion between Python and Scala.

After this fix, for instance, `udf-aggregates_part1.sql` outputs exactly same as `aggregates_part1.sql`:

<details><summary>Diff comparing to 'pgSQL/aggregates_part1.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out
index 51ca1d55869..801735781c7 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out
 -3,7 +3,7

 -- !query 0
-SELECT avg(four) AS avg_1 FROM onek
+SELECT avg(udf(four)) AS avg_1 FROM onek
 -- !query 0 schema
 struct<avg_1:double>
 -- !query 0 output
 -11,7 +11,7  struct<avg_1:double>

 -- !query 1
-SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100
+SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100
 -- !query 1 schema
 struct<avg_32:double>
 -- !query 1 output
 -19,7 +19,7  struct<avg_32:double>

 -- !query 2
-select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest
+select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest
 -- !query 2 schema
 struct<avg_107_943:decimal(10,3)>
 -- !query 2 output
 -27,7 +27,7  struct<avg_107_943:decimal(10,3)>

 -- !query 3
-SELECT sum(four) AS sum_1500 FROM onek
+SELECT sum(udf(four)) AS sum_1500 FROM onek
 -- !query 3 schema
 struct<sum_1500:bigint>
 -- !query 3 output
 -35,7 +35,7  struct<sum_1500:bigint>

 -- !query 4
-SELECT sum(a) AS sum_198 FROM aggtest
+SELECT udf(sum(a)) AS sum_198 FROM aggtest
 -- !query 4 schema
 struct<sum_198:bigint>
 -- !query 4 output
 -43,7 +43,7  struct<sum_198:bigint>

 -- !query 5
-SELECT sum(b) AS avg_431_773 FROM aggtest
+SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest
 -- !query 5 schema
 struct<avg_431_773:double>
 -- !query 5 output
 -51,7 +51,7  struct<avg_431_773:double>

 -- !query 6
-SELECT max(four) AS max_3 FROM onek
+SELECT udf(max(four)) AS max_3 FROM onek
 -- !query 6 schema
 struct<max_3:int>
 -- !query 6 output
 -59,7 +59,7  struct<max_3:int>

 -- !query 7
-SELECT max(a) AS max_100 FROM aggtest
+SELECT max(udf(a)) AS max_100 FROM aggtest
 -- !query 7 schema
 struct<max_100:int>
 -- !query 7 output
 -67,7 +67,7  struct<max_100:int>

 -- !query 8
-SELECT max(aggtest.b) AS max_324_78 FROM aggtest
+SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest
 -- !query 8 schema
 struct<max_324_78:float>
 -- !query 8 output
 -75,237 +75,238  struct<max_324_78:float>

 -- !query 9
-SELECT stddev_pop(b) FROM aggtest
+SELECT stddev_pop(udf(b)) FROM aggtest
 -- !query 9 schema
-struct<stddev_pop(CAST(b AS DOUBLE)):double>
+struct<stddev_pop(CAST(CAST(udf(cast(b as string)) AS FLOAT) AS DOUBLE)):double>
 -- !query 9 output
 131.10703231895047

 -- !query 10
-SELECT stddev_samp(b) FROM aggtest
+SELECT udf(stddev_samp(b)) FROM aggtest
 -- !query 10 schema
-struct<stddev_samp(CAST(b AS DOUBLE)):double>
+struct<CAST(udf(cast(stddev_samp(cast(b as double)) as string)) AS DOUBLE):double>
 -- !query 10 output
 151.38936080399804

 -- !query 11
-SELECT var_pop(b) FROM aggtest
+SELECT var_pop(udf(b)) FROM aggtest
 -- !query 11 schema
-struct<var_pop(CAST(b AS DOUBLE)):double>
+struct<var_pop(CAST(CAST(udf(cast(b as string)) AS FLOAT) AS DOUBLE)):double>
 -- !query 11 output
 17189.053923482323

 -- !query 12
-SELECT var_samp(b) FROM aggtest
+SELECT udf(var_samp(b)) FROM aggtest
 -- !query 12 schema
-struct<var_samp(CAST(b AS DOUBLE)):double>
+struct<CAST(udf(cast(var_samp(cast(b as double)) as string)) AS DOUBLE):double>
 -- !query 12 output
 22918.738564643096

 -- !query 13
-SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest
 -- !query 13 schema
-struct<stddev_pop(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(udf(cast(stddev_pop(cast(cast(b as decimal(38,0)) as double)) as string)) AS DOUBLE):double>
 -- !query 13 output
 131.18117242958306

 -- !query 14
-SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest
 -- !query 14 schema
-struct<stddev_samp(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<stddev_samp(CAST(CAST(CAST(udf(cast(b as string)) AS FLOAT) AS DECIMAL(38,0)) AS DOUBLE)):double>
 -- !query 14 output
 151.47497042966097

 -- !query 15
-SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest
 -- !query 15 schema
-struct<var_pop(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(udf(cast(var_pop(cast(cast(b as decimal(38,0)) as double)) as string)) AS DOUBLE):double>
 -- !query 15 output
 17208.5

 -- !query 16
-SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest
 -- !query 16 schema
-struct<var_samp(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<var_samp(CAST(CAST(udf(cast(cast(b as decimal(38,0)) as string)) AS DECIMAL(38,0)) AS DOUBLE)):double>
 -- !query 16 output
 22944.666666666668

 -- !query 17
-SELECT var_pop(1.0), var_samp(2.0)
+SELECT udf(var_pop(1.0)), var_samp(udf(2.0))
 -- !query 17 schema
-struct<var_pop(CAST(1.0 AS DOUBLE)):double,var_samp(CAST(2.0 AS DOUBLE)):double>
+struct<CAST(udf(cast(var_pop(cast(1.0 as double)) as string)) AS DOUBLE):double,var_samp(CAST(CAST(udf(cast(2.0 as string)) AS DECIMAL(2,1)) AS DOUBLE)):double>
 -- !query 17 output
 0.0    NaN

 -- !query 18
-SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0)))
+SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0)))
 -- !query 18 schema
-struct<stddev_pop(CAST(CAST(3.0 AS DECIMAL(38,0)) AS DOUBLE)):double,stddev_samp(CAST(CAST(4.0 AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<stddev_pop(CAST(CAST(udf(cast(cast(3.0 as decimal(38,0)) as string)) AS DECIMAL(38,0)) AS DOUBLE)):double,stddev_samp(CAST(CAST(CAST(udf(cast(4.0 as string)) AS DECIMAL(2,1)) AS DECIMAL(38,0)) AS DOUBLE)):double>
 -- !query 18 output
 0.0    NaN

 -- !query 19
-select sum(CAST(null AS int)) from range(1,4)
+select sum(udf(CAST(null AS int))) from range(1,4)
 -- !query 19 schema
-struct<sum(CAST(NULL AS INT)):bigint>
+struct<sum(CAST(udf(cast(cast(null as int) as string)) AS INT)):bigint>
 -- !query 19 output
 NULL

 -- !query 20
-select sum(CAST(null AS long)) from range(1,4)
+select sum(udf(CAST(null AS long))) from range(1,4)
 -- !query 20 schema
-struct<sum(CAST(NULL AS BIGINT)):bigint>
+struct<sum(CAST(udf(cast(cast(null as bigint) as string)) AS BIGINT)):bigint>
 -- !query 20 output
 NULL

 -- !query 21
-select sum(CAST(null AS Decimal(38,0))) from range(1,4)
+select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4)
 -- !query 21 schema
-struct<sum(CAST(NULL AS DECIMAL(38,0))):decimal(38,0)>
+struct<sum(CAST(udf(cast(cast(null as decimal(38,0)) as string)) AS DECIMAL(38,0))):decimal(38,0)>
 -- !query 21 output
 NULL

 -- !query 22
-select sum(CAST(null AS DOUBLE)) from range(1,4)
+select sum(udf(CAST(null AS DOUBLE))) from range(1,4)
 -- !query 22 schema
-struct<sum(CAST(NULL AS DOUBLE)):double>
+struct<sum(CAST(udf(cast(cast(null as double) as string)) AS DOUBLE)):double>
 -- !query 22 output
 NULL

 -- !query 23
-select avg(CAST(null AS int)) from range(1,4)
+select avg(udf(CAST(null AS int))) from range(1,4)
 -- !query 23 schema
-struct<avg(CAST(NULL AS INT)):double>
+struct<avg(CAST(udf(cast(cast(null as int) as string)) AS INT)):double>
 -- !query 23 output
 NULL

 -- !query 24
-select avg(CAST(null AS long)) from range(1,4)
+select avg(udf(CAST(null AS long))) from range(1,4)
 -- !query 24 schema
-struct<avg(CAST(NULL AS BIGINT)):double>
+struct<avg(CAST(udf(cast(cast(null as bigint) as string)) AS BIGINT)):double>
 -- !query 24 output
 NULL

 -- !query 25
-select avg(CAST(null AS Decimal(38,0))) from range(1,4)
+select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4)
 -- !query 25 schema
-struct<avg(CAST(NULL AS DECIMAL(38,0))):decimal(38,4)>
+struct<avg(CAST(udf(cast(cast(null as decimal(38,0)) as string)) AS DECIMAL(38,0))):decimal(38,4)>
 -- !query 25 output
 NULL

 -- !query 26
-select avg(CAST(null AS DOUBLE)) from range(1,4)
+select avg(udf(CAST(null AS DOUBLE))) from range(1,4)
 -- !query 26 schema
-struct<avg(CAST(NULL AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(cast(null as double) as string)) AS DOUBLE)):double>
 -- !query 26 output
 NULL

 -- !query 27
-select sum(CAST('NaN' AS DOUBLE)) from range(1,4)
+select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4)
 -- !query 27 schema
-struct<sum(CAST(NaN AS DOUBLE)):double>
+struct<sum(CAST(CAST(udf(cast(NaN as string)) AS STRING) AS DOUBLE)):double>
 -- !query 27 output
 NaN

 -- !query 28
-select avg(CAST('NaN' AS DOUBLE)) from range(1,4)
+select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4)
 -- !query 28 schema
-struct<avg(CAST(NaN AS DOUBLE)):double>
+struct<avg(CAST(CAST(udf(cast(NaN as string)) AS STRING) AS DOUBLE)):double>
 -- !query 28 output
 NaN

 -- !query 30
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('Infinity'), ('1')) v(x)
 -- !query 30 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(CAST(udf(cast(x as string)) AS STRING) AS DOUBLE)):double,var_pop(CAST(CAST(udf(cast(x as string)) AS STRING) AS DOUBLE)):double>
 -- !query 30 output
 Infinity       NaN

 -- !query 31
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('Infinity'), ('Infinity')) v(x)
 -- !query 31 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(CAST(udf(cast(x as string)) AS STRING) AS DOUBLE)):double,var_pop(CAST(CAST(udf(cast(x as string)) AS STRING) AS DOUBLE)):double>
 -- !query 31 output
 Infinity       NaN

 -- !query 32
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('-Infinity'), ('Infinity')) v(x)
 -- !query 32 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(CAST(udf(cast(x as string)) AS STRING) AS DOUBLE)):double,var_pop(CAST(CAST(udf(cast(x as string)) AS STRING) AS DOUBLE)):double>
 -- !query 32 output
 NaN    NaN

 -- !query 33
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE)))
 FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x)
 -- !query 33 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(cast(x as double) as string)) AS DOUBLE)):double,CAST(udf(cast(var_pop(cast(x as double)) as string)) AS DOUBLE):double>
 -- !query 33 output
 1.00000005E8   2.5

 -- !query 34
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE)))
 FROM (VALUES (7000000000005), (7000000000007)) v(x)
 -- !query 34 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(cast(x as double) as string)) AS DOUBLE)):double,CAST(udf(cast(var_pop(cast(x as double)) as string)) AS DOUBLE):double>
 -- !query 34 output
 7.000000000006E12      1.0

 -- !query 35
-SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest
+SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest
 -- !query 35 schema
-struct<covar_pop(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double,covar_samp(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double>
+struct<CAST(udf(cast(covar_pop(cast(b as double), cast(cast(udf(cast(a as string)) as int) as double)) as string)) AS DOUBLE):double,covar_samp(CAST(CAST(udf(cast(b as string)) AS FLOAT) AS DOUBLE), CAST(a AS DOUBLE)):double>
 -- !query 35 output
 653.6289553875104      871.5052738500139

 -- !query 36
-SELECT corr(b, a) FROM aggtest
+SELECT corr(b, udf(a)) FROM aggtest
 -- !query 36 schema
-struct<corr(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double>
+struct<corr(CAST(b AS DOUBLE), CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double>
 -- !query 36 output
 0.1396345165178734

 -- !query 37
-SELECT count(four) AS cnt_1000 FROM onek
+SELECT count(udf(four)) AS cnt_1000 FROM onek
 -- !query 37 schema
 struct<cnt_1000:bigint>
 -- !query 37 output
 -313,7 +314,7  struct<cnt_1000:bigint>

 -- !query 38
-SELECT count(DISTINCT four) AS cnt_4 FROM onek
+SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek
 -- !query 38 schema
 struct<cnt_4:bigint>
 -- !query 38 output
 -321,10 +322,10  struct<cnt_4:bigint>

 -- !query 39
-select ten, count(*), sum(four) from onek
+select ten, udf(count(*)), sum(udf(four)) from onek
 group by ten order by ten
 -- !query 39 schema
-struct<ten:int,count(1):bigint,sum(four):bigint>
+struct<ten:int,CAST(udf(cast(count(1) as string)) AS BIGINT):bigint,sum(CAST(udf(cast(four as string)) AS INT)):bigint>
 -- !query 39 output
 0      100     100
 1      100     200
 -339,10 +340,10  struct<ten:int,count(1):bigint,sum(four):bigint>

 -- !query 40
-select ten, count(four), sum(DISTINCT four) from onek
+select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek
 group by ten order by ten
 -- !query 40 schema
-struct<ten:int,count(four):bigint,sum(DISTINCT four):bigint>
+struct<ten:int,count(CAST(udf(cast(four as string)) AS INT)):bigint,CAST(udf(cast(sum(distinct cast(four as bigint)) as string)) AS BIGINT):bigint>
 -- !query 40 output
 0      100     2
 1      100     4
 -357,11 +358,11  struct<ten:int,count(four):bigint,sum(DISTINCT four):bigint>

 -- !query 41
-select ten, sum(distinct four) from onek a
+select ten, udf(sum(distinct four)) from onek a
 group by ten
-having exists (select 1 from onek b where sum(distinct a.four) = b.four)
+having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four)
 -- !query 41 schema
-struct<ten:int,sum(DISTINCT four):bigint>
+struct<ten:int,CAST(udf(cast(sum(distinct cast(four as bigint)) as string)) AS BIGINT):bigint>
 -- !query 41 output
 0      2
 2      2
 -374,23 +375,23  struct<ten:int,sum(DISTINCT four):bigint>
 select ten, sum(distinct four) from onek a
 group by ten
 having exists (select 1 from onek b
-               where sum(distinct a.four + b.four) = b.four)
+               where sum(distinct a.four + b.four) = udf(b.four))
 -- !query 42 schema
 struct<>
 -- !query 42 output
 org.apache.spark.sql.AnalysisException

 Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(b.`four` AS BIGINT))]
+Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(CAST(udf(cast(four as string)) AS INT) AS BIGINT))]
 Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))];

 -- !query 43
 select
-  (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))
+  (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))))
 from tenk1 o
 -- !query 43 schema
 struct<>
 -- !query 43 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 63
+cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 67
```
</p>
</details>

## How was this patch tested?

Manually tested.

Closes #25130 from HyukjinKwon/SPARK-28359.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-17 21:49:43 +08:00
nooberfsh 1134faecf4 [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset
## What changes were proposed in this pull request?

Add 4 additional agg to KeyValueGroupedDataset

## How was this patch tested?

New test in DatasetSuite for typed aggregation

Closes #24993 from nooberfsh/sqlagg.

Authored-by: nooberfsh <nooberfsh@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-16 16:35:04 -07:00
Yuming Wang 71882f119e [SPARK-28343][FOLLOW-UP][SQL][TEST] Enable spark.sql.function.preferIntegralDivision for PostgreSQL testing
## What changes were proposed in this pull request?

This PR enables `spark.sql.function.preferIntegralDivision` for PostgreSQL testing.

## How was this patch tested?

N/A

Closes #25170 from wangyum/SPARK-28343-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-16 08:46:01 -07:00
Gabor Somogyi 113f62dd8c [SPARK-27485][FOLLOWUP] Do not reduce the number of partitions for repartition in adaptive execution - fix compilation
## What changes were proposed in this pull request?

PR builder failed with the following error:
```
[error] /home/jenkins/workspace/SparkPullRequestBuilder/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala:714: wrong number of arguments for pattern org.apache.spark.sql.execution.exchange.ShuffleExchangeExec(outputPartitioning: org.apache.spark.sql.catalyst.plans.physical.Partitioning,child: org.apache.spark.sql.execution.SparkPlan,canChangeNumPartitions: Boolean)
[error]                ShuffleExchangeExec(HashPartitioning(leftPartitioningExpressions, _), _), _),
[error]                                   ^
[error] /home/jenkins/workspace/SparkPullRequestBuilder/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala:716: wrong number of arguments for pattern org.apache.spark.sql.execution.exchange.ShuffleExchangeExec(outputPartitioning: org.apache.spark.sql.catalyst.plans.physical.Partitioning,child: org.apache.spark.sql.execution.SparkPlan,canChangeNumPartitions: Boolean)
[error]                ShuffleExchangeExec(HashPartitioning(rightPartitioningExpressions, _), _), _)) =>
[error]                                   ^
```

## How was this patch tested?

Existing unit test.

Closes #25171 from gaborgsomogyi/SPARK-27485.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: herman <herman@databricks.com>
2019-07-16 12:56:13 +02:00
Yuming Wang f74ad3d700 [SPARK-28129][SQL][TEST] Port float8.sql
## What changes were proposed in this pull request?

This PR is to port float8.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float8.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/float8.out

When porting the test cases, found six PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28060](https://issues.apache.org/jira/browse/SPARK-28060): Double type can not accept some special inputs
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027): Spark SQL does not support prefix operator `` and `|/`
[SPARK-28061](https://issues.apache.org/jira/browse/SPARK-28061): Support for converting float to binary format
[SPARK-23906](https://issues.apache.org/jira/browse/SPARK-23906): Support Truncate number
[SPARK-28134](https://issues.apache.org/jira/browse/SPARK-28134): Missing Trigonometric Functions

Also, found two bug:
[SPARK-28024](https://issues.apache.org/jira/browse/SPARK-28024): Incorrect value when out of range
[SPARK-28135](https://issues.apache.org/jira/browse/SPARK-28135): ceil/ceiling/floor/power returns incorrect values

Also, found four inconsistent behavior:
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Spark SQL insert bad inputs to NULL
[SPARK-28028](https://issues.apache.org/jira/browse/SPARK-28028): Cast numeric to integral type need round
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Spark SQL returns NULL when dividing by zero
[SPARK-28007](https://issues.apache.org/jira/browse/SPARK-28007):  Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres

## How was this patch tested?

N/A

Closes #24931 from wangyum/SPARK-28129.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-16 19:31:20 +09:00
Carson Wang d1a1376029 [SPARK-28356][SQL] Do not reduce the number of partitions for repartition in adaptive execution
## What changes were proposed in this pull request?
Adaptive execution reduces the number of post-shuffle partitions at runtime, even for shuffles caused by repartition. However, the user likely wants to get the desired number of partition when he calls repartition even in adaptive execution. This PR adds an internal config to control this and by default adaptive execution will not change the number of post-shuffle partition for repartition.

## How was this patch tested?
New tests added.

Closes #25121 from carsonwang/AE_repartition.

Authored-by: Carson Wang <carson.wang@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-16 17:35:46 +08:00
herman 421d9d56ef [SPARK-27485] EnsureRequirements.reorder should handle duplicate expressions gracefully
## What changes were proposed in this pull request?
When reordering joins EnsureRequirements only checks if all the join keys are present in the partitioning expression seq. This is problematic when the joins keys and and partitioning expressions both contain duplicates but not the same number of duplicates for each expression, e.g. `Seq(a, a, b)` vs `Seq(a, b, b)`. This fails with an index lookup failure in the `reorder` function.

This PR fixes this removing the equality checking logic from the `reorderJoinKeys` function, and by doing the multiset equality in the `reorder` function while building the reordered key sequences.

## How was this patch tested?
Added a unit test to the `PlannerSuite` and added an integration test to `JoinSuite`

Closes #25167 from hvanhovell/SPARK-27485.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-16 17:09:52 +08:00
Yuming Wang 6926849247 [SPARK-28395][SQL] Division operator support integral division
## What changes were proposed in this pull request?

PostgreSQL, Teradata, SQL Server, DB2 and Presto perform integral division with the `/` operator.
But Oracle, Vertica, Hive, MySQL and MariaDB perform fractional division with the `/` operator.

This pr add a flag(`spark.sql.function.preferIntegralDivision`) to control whether to use integral division with the `/` operator.

Examples:

**PostgreSQL**:
```sql
postgres=# select substr(version(), 0, 16), cast(10 as int) / cast(3 as int), cast(10.1 as float8) / cast(3 as int), cast(10 as int) / cast(3.1 as float8), cast(10.1 as float8)/cast(3.1 as float8);
     substr      | ?column? |     ?column?     |    ?column?     |     ?column?
-----------------+----------+------------------+-----------------+------------------
 PostgreSQL 11.3 |        3 | 3.36666666666667 | 3.2258064516129 | 3.25806451612903
(1 row)
```
**SQL Server**:
```sql
1> select cast(10 as int) / cast(3 as int), cast(10.1 as float) / cast(3 as int), cast(10 as int) / cast(3.1 as float), cast(10.1 as float)/cast(3.1 as float);
2> go

----------- ------------------------ ------------------------ ------------------------
          3       3.3666666666666667        3.225806451612903        3.258064516129032

(1 rows affected)
```
**DB2**:
```sql
[db2inst12f3c821d36b7 ~]$ db2 "select cast(10 as int) / cast(3 as int), cast(10.1 as double) / cast(3 as int), cast(10 as int) / cast(3.1 as double), cast(10.1 as double)/cast(3.1 as double) from table (sysproc.env_get_inst_info())"

1           2                        3                        4
----------- ------------------------ ------------------------ ------------------------
          3   +3.36666666666667E+000   +3.22580645161290E+000   +3.25806451612903E+000

  1 record(s) selected.
```
**Presto**:
```sql
presto> select cast(10 as int) / cast(3 as int), cast(10.1 as double) / cast(3 as int), cast(10 as int) / cast(3.1 as double), cast(10.1 as double)/cast(3.1 as double);
 _col0 |       _col1        |       _col2       |       _col3
-------+--------------------+-------------------+-------------------
     3 | 3.3666666666666667 | 3.225806451612903 | 3.258064516129032
(1 row)
```
**Teradata**:
![image](https://user-images.githubusercontent.com/5399861/61200701-e97d5380-a714-11e9-9a1d-57fd99d38c8d.png)

**Oracle**:
```sql
SQL> select 10 / 3 from dual;

      10/3
----------
3.33333333
```
**Vertica**
```sql
dbadmin=> select version(), cast(10 as int) / cast(3 as int), cast(10.1 as float8) / cast(3 as int), cast(10 as int) / cast(3.1 as float8), cast(10.1 as float8)/cast(3.1 as float8);
              version               |       ?column?       |     ?column?     |    ?column?     |     ?column?
------------------------------------+----------------------+------------------+-----------------+------------------
 Vertica Analytic Database v9.1.1-0 | 3.333333333333333333 | 3.36666666666667 | 3.2258064516129 | 3.25806451612903
(1 row)
```
**Hive**:
```sql
hive> select cast(10 as int) / cast(3 as int), cast(10.1 as double) / cast(3 as int), cast(10 as int) / cast(3.1 as double), cast(10.1 as double)/cast(3.1 as double);
OK
3.3333333333333335	3.3666666666666667	3.225806451612903	3.258064516129032
Time taken: 0.143 seconds, Fetched: 1 row(s)
```
**MariaDB**:
```sql
MariaDB [(none)]> select version(), cast(10 as int) / cast(3 as int), cast(10.1 as double) / cast(3 as int), cast(10 as int) / cast(3.1 as double), cast(10.1 as double)/cast(3.1 as double);
+--------------------------------------+----------------------------------+---------------------------------------+---------------------------------------+------------------------------------------+
| version()                            | cast(10 as int) / cast(3 as int) | cast(10.1 as double) / cast(3 as int) | cast(10 as int) / cast(3.1 as double) | cast(10.1 as double)/cast(3.1 as double) |
+--------------------------------------+----------------------------------+---------------------------------------+---------------------------------------+------------------------------------------+
| 10.4.6-MariaDB-1:10.4.6+maria~bionic |                           3.3333 |                    3.3666666666666667 |                     3.225806451612903 |                        3.258064516129032 |
+--------------------------------------+----------------------------------+---------------------------------------+---------------------------------------+------------------------------------------+
1 row in set (0.000 sec)
```
**MySQL**:
```sql
mysql>  select version(), 10 / 3, 10 / 3.1, 10.1 / 3, 10.1 / 3.1;
+-----------+--------+----------+----------+------------+
| version() | 10 / 3 | 10 / 3.1 | 10.1 / 3 | 10.1 / 3.1 |
+-----------+--------+----------+----------+------------+
| 8.0.16    | 3.3333 |   3.2258 |  3.36667 |    3.25806 |
+-----------+--------+----------+----------+------------+
1 row in set (0.00 sec)
```
## How was this patch tested?

unit tests

Closes #25158 from wangyum/SPARK-28395.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-16 15:43:15 +08:00
Liang-Chi Hsieh b94fa979ef [SPARK-28345][SQL][PYTHON] PythonUDF predicate should be able to pushdown to join
## What changes were proposed in this pull request?

A `Filter` predicate using `PythonUDF` can't be push down into join condition, currently. A predicate like that should be able to push down to join condition. For `PythonUDF`s that can't be evaluated in join condition, `PullOutPythonUDFInJoinCondition` will pull them out later.

An example like:

```scala
val pythonTestUDF = TestPythonUDF(name = "udf")

val left = Seq((1, 2), (2, 3)).toDF("a", "b")
val right = Seq((1, 2), (3, 4)).toDF("c", "d")
val df = left.crossJoin(right).where(pythonTestUDF($"a") === pythonTestUDF($"c"))
```

Query plan before the PR:
```
== Physical Plan ==
*(3) Project [a#2121, b#2122, c#2132, d#2133]
+- *(3) Filter (pythonUDF0#2142 = pythonUDF1#2143)
   +- BatchEvalPython [udf(a#2121), udf(c#2132)], [pythonUDF0#2142, pythonUDF1#2143]
      +- BroadcastNestedLoopJoin BuildRight, Cross
         :- *(1) Project [_1#2116 AS a#2121, _2#2117 AS b#2122]
         :  +- LocalTableScan [_1#2116, _2#2117]
         +- BroadcastExchange IdentityBroadcastMode
            +- *(2) Project [_1#2127 AS c#2132, _2#2128 AS d#2133]
               +- LocalTableScan [_1#2127, _2#2128]
```

Query plan after the PR:
```
== Physical Plan ==
*(3) Project [a#2121, b#2122, c#2132, d#2133]
+- *(3) BroadcastHashJoin [pythonUDF0#2142], [pythonUDF0#2143], Cross, BuildRight
   :- BatchEvalPython [udf(a#2121)], [pythonUDF0#2142]
   :  +- *(1) Project [_1#2116 AS a#2121, _2#2117 AS b#2122]
   :     +- LocalTableScan [_1#2116, _2#2117]
   +- BroadcastExchange HashedRelationBroadcastMode(List(input[2, string, true]))
      +- BatchEvalPython [udf(c#2132)], [pythonUDF0#2143]
         +- *(2) Project [_1#2127 AS c#2132, _2#2128 AS d#2133]
            +- LocalTableScan [_1#2127, _2#2128]
```

After this PR, the join can use `BroadcastHashJoin`, instead of `BroadcastNestedLoopJoin`.

## How was this patch tested?

Added tests.

Closes #25106 from viirya/pythonudf-join-condition.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-16 16:15:49 +09:00
Maxim Gekk 8e26d4d616 [SPARK-28408][SQL][TEST] Restrict test values for DateType, TimestampType and CalendarIntervalType
## What changes were proposed in this pull request?

Existing random generators in tests produce wide ranges of values that can be out of supported ranges for:
- `DateType`, the valid range is `[0001-01-01, 9999-12-31]`
- `TimestampType` supports values in `[0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z]`
- `CalendarIntervalType` should define intervals for the ranges above.

Dates and timestamps produced by random literal generators are usually out of valid ranges for those types. And tests just check invalid values or values caused by arithmetic overflow.

In the PR, I propose to restrict tested pseudo-random values by valid ranges of `DateType`, `TimestampType` and `CalendarIntervalType`. This should allow to check valid values in test, and avoid wasting time on a priori invalid inputs.

## How was this patch tested?

The changes were checked by `DateExpressionsSuite` and modified `DateTimeUtils.dateAddMonths`:
```Scala
  def dateAddMonths(days: SQLDate, months: Int): SQLDate = {
    localDateToDays(LocalDate.ofEpochDay(days).plusMonths(months))
  }
```

Closes #25166 from MaxGekk/datetime-lit-random-gen.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-15 20:42:33 -07:00
shivsood d8996fd940 [SPARK-28152][SQL] Mapped ShortType to SMALLINT and FloatType to REAL for MsSqlServerDialect
## What changes were proposed in this pull request?
This PR aims to correct mappings in `MsSqlServerDialect`. `ShortType` is mapped to `SMALLINT` and `FloatType` is mapped to `REAL` per [JBDC mapping]( https://docs.microsoft.com/en-us/sql/connect/jdbc/using-basic-data-types?view=sql-server-2017) respectively.

ShortType and FloatTypes are not correctly mapped to right JDBC types when using JDBC connector. This results in tables and spark data frame being created with unintended types. The issue was observed when validating against SQLServer.

Refer [JBDC mapping]( https://docs.microsoft.com/en-us/sql/connect/jdbc/using-basic-data-types?view=sql-server-2017  ) for guidance on mappings between SQLServer, JDBC and Java. Note that java "Short" type should be mapped to JDBC "SMALLINT" and java Float should be mapped to JDBC "REAL".

Some example issue that can happen because of wrong mappings
    - Write from df with column type results in a SQL table of with column type as INTEGER as opposed to SMALLINT.Thus a larger table that expected.
    - Read results in a dataframe with type INTEGER as opposed to ShortType

- ShortType has a problem in both the the write and read path
- FloatTypes only have an issue with read path. In the write path Spark data type 'FloatType' is correctly mapped to JDBC equivalent data type 'Real'. But in the read path when JDBC data types need to be converted to Catalyst data types ( getCatalystType) 'Real' gets incorrectly gets mapped to 'DoubleType' rather than 'FloatType'.

Refer #28151 which contained this fix as one part of a larger PR.  Following PR #28151 discussion it was decided to file seperate PRs for each of the fixes.

## How was this patch tested?
UnitTest added in JDBCSuite.scala and these were tested.
Integration test updated and passed in MsSqlServerDialect.scala
E2E test done with SQLServer

Closes #25146 from shivsood/float_short_type_fix.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-15 12:12:36 -07:00
Gabor Somogyi 8f7ccc5e9c [SPARK-28404][SS] Fix negative timeout value in RateStreamContinuousPartitionReader
## What changes were proposed in this pull request?

`System.currentTimeMillis` read two times in a loop in `RateStreamContinuousPartitionReader`. If the test machine is slow enough and it spends quite some time between the `while` condition check and the `Thread.sleep` then the timeout value is negative and throws `IllegalArgumentException`.

In this PR I've fixed this issue.

## How was this patch tested?

Existing unit tests.

Closes #25162 from gaborgsomogyi/SPARK-28404.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-15 11:01:03 -07:00
Yesheng Ma 2f3997fddc [SPARK-28306][SQL][FOLLOWUP] Fix NormalizeFloatingNumbers rule idempotence for equi-join with <=> predicates
## What changes were proposed in this pull request?
Idempotence of the `NormalizeFloatingNumbers` rule was broken due to the implementation of `ExtractEquiJoinKeys`. There is no reason that we don't remove `EqualNullSafe` join keys from an equi-join's `otherPredicates`.

## How was this patch tested?
A new UT.

Closes #25126 from yeshengm/spark-28306.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-15 10:38:49 -07:00
朱夷 8ecbb67b3b [SPARK-28311][SQL] Fix STS OpenSession failed return wrong origin PROTOCOL_VERSION
## What changes were proposed in this pull request?

For Thrift server, It's downward compatible. Such as if a PROTOCOL_VERSION_V7 client connect to a  PROTOCOL_VERSION_V8 server, when OpenSession, server will change his response's protocol version to min of (client and server).
`TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION,`
 `       req.getClient_protocol());`
then set it to OpenSession's response.

But if OpenSession failed , it won't execute behavior of reset response's protocol_version.
Then it will return server's origin protocol version.
Finally client will get en error as below:
![image](https://user-images.githubusercontent.com/46485123/61023164-54f4b780-a3db-11e9-8c49-60217b36287b.png)
Since we write a wrong database,, OpenSession failed, right protocol version haven't been rest.

## How was this patch tested?

Since I really don't know how to write unit test about this, so I build a jar with this PR,and retry the error above, then it will return a reasonable Error of DB not found :
![image](https://user-images.githubusercontent.com/46485123/61023923-67242500-a3de-11e9-8e98-8f391a038480.png)

Closes #25083 from AngersZhuuuu/SPARK-28311.

Authored-by: 朱夷 <zhuyi01@corp.netease.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-15 09:35:56 -05:00
Maxim Gekk f241fc7776 [SPARK-28389][SQL] Use Java 8 API in add_months
## What changes were proposed in this pull request?

In the PR, I propose to use the `plusMonths()` method of `LocalDate` to add months to a date. This method adds the specified amount to the months field of `LocalDate` in three steps:
1. Add the input months to the month-of-year field
2. Check if the resulting date would be invalid
3. Adjust the day-of-month to the last valid day if necessary

The difference between current behavior and propose one is in handling the last day of month in the original date. For example, adding 1 month to `2019-02-28` will produce `2019-03-28` comparing to the current implementation where the result is `2019-03-31`.

The proposed behavior is implemented in MySQL and PostgreSQL.

## How was this patch tested?

By existing test suites `DateExpressionsSuite`, `DateFunctionsSuite` and `DateTimeUtilsSuite`.

Closes #25153 from MaxGekk/add-months.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-15 20:49:39 +08:00
HyukjinKwon a7a02a86ad [SPARK-28392][SQL][TESTS] Add traits for UDF and PostgreSQL tests to share initialization
## What changes were proposed in this pull request?

This PR adds some traits so that we can deduplicate initialization stuff for each type of test case. For instance, see [SPARK-28343](https://issues.apache.org/jira/browse/SPARK-28343).

It's a little bit overkill but I think it will make adding test cases easier and cause less confusions.

This PR adds both:

```
  private trait PgSQLTest
  private trait UDFTest
```

To indicate and share the logics related to each combination of test types.

## How was this patch tested?

Manually tested.

Closes #25155 from HyukjinKwon/SPARK-28392.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-15 16:20:09 +09:00
Yuming Wang 72cc853092 [SPARK-28384][SQL][TEST] Port select_distinct.sql
## What changes were proposed in this pull request?

This PR is to port select.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_distinct.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/select_distinct.out

When porting the test cases, found one PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28010](https://issues.apache.org/jira/browse/SPARK-28010): Support ORDER BY ... USING syntax

## How was this patch tested?

N/A

Closes #25150 from wangyum/SPARK-28384.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-14 21:55:11 -07:00
Yuming Wang e238ebe9b0 [SPARK-28387][SQL][TEST] Port select_having.sql
## What changes were proposed in this pull request?

This PR is to port select.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/select_having.out

When porting the test cases, found one bug:
[SPARK-28386](https://issues.apache.org/jira/browse/SPARK-28386): Cannot resolve ORDER BY columns with GROUP BY and HAVING

## How was this patch tested?

N/A

Closes #25151 from wangyum/SPARK-28387.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-14 21:21:09 -07:00
Tony Zhang a2f71a8d85 [SPARK-28133][SQL] Add acosh/asinh/atanh functions to SQL
## What changes were proposed in this pull request?

Adding support to hyperbolic functions like asinh\acosh\atanh in spark SQL.
Feature parity: https://www.postgresql.org/docs/12/functions-math.html#FUNCTIONS-MATH-HYP-TABLE

The followings are the diffence from PostgreSQL.
```
spark-sql> SELECT acosh(0);     (PostgreSQL returns `ERROR:  input is out of range`)
NaN

spark-sql> SELECT atanh(2);     (PostgreSQL returns `ERROR:  input is out of range`)
NaN
```

Teradata has similar behavior as PostgreSQL with out of range input float values - It outputs **Invalid Input: numeric value within range only.**

These newly added asinh/acosh/atanh handles special input(NaN, +-Infinity) in the same way as existing cos/sin/tan/acos/asin/atan in spark. For which input value range is not (-∞, ∞)):
out of range float values: Spark returns NaN and PostgreSQL shows input is out of range
NaN: Spark returns NaN, PostgreSQL also returns NaN
Infinity: Spark return NaN, PostgreSQL shows input is out of range

## How was this patch tested?

```
spark.sql("select asinh(xx)")
spark.sql("select acosh(xx)")
spark.sql("select atanh(xx)")

./build/sbt "testOnly org.apache.spark.sql.MathFunctionsSuite"
./build/sbt "testOnly org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite"
```

Closes #25041 from Tonix517/SPARK-28133.

Authored-by: Tony Zhang <tony.zhang@uber.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-14 20:41:45 -07:00
Liang-Chi Hsieh 591de42351 [SPARK-28381][PYSPARK] Upgraded version of Pyrolite to 4.30
## What changes were proposed in this pull request?

This upgraded to a newer version of Pyrolite. Most updates [1] in the newer version are for dotnot. For java, it includes a bug fix to Unpickler regarding cleaning up Unpickler memo, and support of protocol 5.

After upgrading, we can remove the fix at SPARK-27629 for the bug in Unpickler.

[1] https://github.com/irmen/Pyrolite/compare/pyrolite-4.23...master

## How was this patch tested?

Manually tested on Python 3.6 in local on existing tests.

Closes #25143 from viirya/upgrade-pyrolite.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-15 12:29:58 +09:00
Jungtaek Lim (HeartSaVioR) 7548a8826d [SPARK-28199][SS] Move Trigger implementations to Triggers.scala and avoid exposing these to the end users
## What changes were proposed in this pull request?

This patch proposes moving all Trigger implementations to `Triggers.scala`, to avoid exposing these implementations to the end users and let end users only deal with `Trigger.xxx` static methods. This fits the intention of deprecation of `ProcessingTIme`, and we agree to move others without deprecation as this patch will be shipped in major version (Spark 3.0.0).

## How was this patch tested?

UTs modified to work with newly introduced class.

Closes #24996 from HeartSaVioR/SPARK-28199.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-14 14:46:01 -05:00
Yuming Wang 76079fab5c [SPARK-28343][SQL][TEST] Enabling cartesian product and ansi mode for PostgreSQL testing
## What changes were proposed in this pull request?

This pr enables `spark.sql.crossJoin.enabled` and `spark.sql.parser.ansi.enabled` for PostgreSQL test.

## How was this patch tested?

manual tests:
Run `test.sql` in [pgSQL](https://github.com/apache/spark/tree/master/sql/core/src/test/resources/sql-tests/inputs/pgSQL) directory and in [inputs](https://github.com/apache/spark/tree/master/sql/core/src/test/resources/sql-tests/inputs) directory:
```sql
cat <<EOF > test.sql
create or replace temporary view t1 as
select * from (values(1), (2)) as v (val);

create or replace temporary view t2 as
select * from (values(2), (1)) as v (val);

select t1.*, t2.* from t1 join t2;
EOF
```

Closes #25109 from wangyum/SPARK-28343.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-13 23:37:58 -07:00
Marcelo Vanzin 7f9da2b7f8 [SPARK-28371][SQL] Make Parquet "StartsWith" filter null-safe
Parquet may call the filter with a null value to check whether nulls are
accepted. While it seems Spark avoids that path in Parquet with 1.10, in
1.11 that causes Spark unit tests to fail.

Tested with Parquet 1.11 (and new unit test).

Closes #25140 from vanzin/SPARK-28371.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-13 11:38:54 -07:00
Jungtaek Lim (HeartSaVioR) b5a9baa19c [SPARK-28247][SS] Fix flaky test "query without test harness" on ContinuousSuite
## What changes were proposed in this pull request?

This patch fixes the flaky test "query without test harness" on ContinuousSuite, via adding some more gaps on waiting query to commit the epoch which writes output rows.

The observation of this issue is below (injected some debug logs to get them):

```
reader creation time                                   1562225320210
epoch 1 launched                                       1562225320593 (+380ms from reader creation time)
epoch 13 launched                                      1562225321702 (+1.5s from reader creation time)
partition reader creation time                         1562225321715 (+1.5s from reader creation time)

next read time for first next call                     1562225321210 (+1s from reader creation time)
first next called in partition reader                  1562225321746 (immediately after creation of partition reader)
wait finished in next called in partition reader       1562225321746 (no wait)

second next called in partition reader                 1562225321747 (immediately after first next())

epoch 0 commit started                                 1562225321861

writing rows (0, 1) (belong to epoch 13)               1562225321866 (+100ms after first next())

wait start in waitForRateSourceTriggers(2)             1562225322059

next read time for second next call                    1562225322210 (+1s from previous "next read time")
wait finished in next called in partition reader       1562225322211 (+450ms wait)

writing rows (2, 3) (belong to epoch 13)               1562225322211 (immediately after next())

epoch 14 launched                                      1562225322246

desired wait time in waitForRateSourceTriggers(2)      1562225322510 (+2.3s from reader creation time)

epoch 12 committed                                     1562225323034
```

These rows were written within desired wait time, but the epoch 13 couldn't be committed within it. Interestingly, epoch 12 was lucky to be committed within a gap between finished waiting in waitForRateSourceTriggers and query.stop() - but even suppose the rows were written in epoch 12, it would be just in luck and epoch should be committed within desired wait time.

This patch modifies Rate continuous stream to track the highest committed value, so that test can wait until desired value is reported to the stream as committed.

This patch also modifies Rate continuous stream to track the timestamp at stream gets the first committed offset, and let `waitForRateSourceTriggers` use the timestamp. This also relies on waiting for specific period, but safer approach compared to current based on the observation above. Based on the change, this patch saves couple of seconds in test time.

## How was this patch tested?

10 sequential test runs succeeded locally.

Closes #25048 from HeartSaVioR/SPARK-28247.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-13 12:11:06 -05:00
gatorsmile 60b89cf809 [SPARK-28361][SQL][TEST] Test equality of generated code with id in class name
A code gen test in WholeStageCodeGenSuite was flaky because it used the codegen metrics class to test if the generated code for equivalent plans was identical under a particular flag. This patch switches the test to compare the generated code directly.

N/A

Closes #25131 from gatorsmile/WholeStageCodegenSuite.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-12 16:12:52 -07:00
Yuming Wang 687dd4eb55 [SPARK-28260][SQL] Add CLOSED state to ExecutionState
## What changes were proposed in this pull request?

The `ThriftServerTab` displays a FINISHED state when the operation finishes execution, but quite often it still takes a lot of time to fetch the results. OperationState has state CLOSED for when after the iterator is closed. This PR add CLOSED state to ExecutionState, and override the `close()` in SparkExecuteStatementOperation, SparkGetColumnsOperation, SparkGetSchemasOperation and SparkGetTablesOperation.

## How was this patch tested?

manual tests
1. Add `Thread.sleep(10000)` before [SparkExecuteStatementOperation.scala#L112](b2e7677f4d/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala (L112))
2. Switch to `ThriftServerTab`:
![image](https://user-images.githubusercontent.com/5399861/60809590-9dcf2500-a1bd-11e9-826e-33729bb97daf.png)
3. After a while:
![image](https://user-images.githubusercontent.com/5399861/60809719-e850a180-a1bd-11e9-9a6a-546146e626ab.png)

Closes #25062 from wangyum/SPARK-28260.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-12 10:31:28 -07:00
Peter Toth 1a26126d8c [SPARK-28228][SQL] Fix substitution order of nested WITH clauses
## What changes were proposed in this pull request?

This PR adds compatibility of handling a `WITH` clause within another `WITH` cause. Before this PR these queries retuned `1` while after this PR they return `2` as PostgreSQL does:
```
WITH
  t AS (SELECT 1),
  t2 AS (
    WITH t AS (SELECT 2)
    SELECT * FROM t
  )
SELECT * FROM t2
```
```
WITH t AS (SELECT 1)
SELECT (
  WITH t AS (SELECT 2)
  SELECT * FROM t
)
```
As this is an incompatible change, the PR introduces the `spark.sql.legacy.cte.substitution.enabled` flag as an option to restore old behaviour.

## How was this patch tested?

Added new UTs.

Closes #25029 from peter-toth/SPARK-28228.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-12 07:17:33 -07:00
Peter Toth fe22faa7fa [SPARK-28034][SQL][TEST] Port with.sql
## What changes were proposed in this pull request?

This PR is to port with.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/with.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/with.out

When porting the test cases, found 7 PostgreSQL specific features that do not exist in Spark SQL:

- [SPARK-19799](https://issues.apache.org/jira/browse/SPARK-19799) Support WITH clause in subqueries
- [SPARK-24497](https://issues.apache.org/jira/browse/SPARK-24497) Support recursive SQL query
- [SPARK-28297](https://issues.apache.org/jira/browse/SPARK-28297) Handling outer links in CTE subquery expressions
- [SPARK-28296](https://issues.apache.org/jira/browse/SPARK-28296) Improved VALUES support
- [SPARK-28146](https://issues.apache.org/jira/browse/SPARK-28146) Support IS OF type predicate
- [SPARK-28147](https://issues.apache.org/jira/browse/SPARK-28147) Support RETURNING clause
- [SPARK-27878](https://issues.apache.org/jira/browse/SPARK-27878) Support ARRAY(sub-SELECT) expressions

Also, found one inconsistent behavior:
- [SPARK-28299](https://issues.apache.org/jira/browse/SPARK-28299) Evaluation of multiple CTE uses

Also, added the following notes:
- Spark SQL doesn't support DELETE statement
- Spark SQL doesn't support UPDATE statement
- Spark SQL doesn't support RULEs
- Spark SQL doesn't support UNIQUE constraints
- Spark SQL doesn't support ON CONFLICT clause
- Spark SQL doesn't support TRIGGERs
- Spark SQL doesn't support INHERITS clause

## How was this patch tested?

N/A

Closes #24860 from peter-toth/SPARK-28034.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-11 22:55:33 -07:00
wangguangxin.cn 42b80ae128 [SPARK-28257][SQL] Use ConfigEntry for hardcoded configs in SQL
## What changes were proposed in this pull request?

There are some hardcoded configs, using config entry to replace them.

## How was this patch tested?

Existing UT

Closes #25059 from WangGuangxin/ConfigEntry.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-11 22:36:07 -07:00
HyukjinKwon 27e41d65f1 [SPARK-28270][TEST-MAVEN][FOLLOW-UP][SQL][PYTHON][TESTS] Avoid cast input of UDF as double in the failed test in udf-aggregate_part1.sql
## What changes were proposed in this pull request?

It still can be flaky on certain environments due to float limitation described at https://github.com/apache/spark/pull/25110 . See https://github.com/apache/spark/pull/25110#discussion_r302735905

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/6584/testReport/org.apache.spark.sql/SQLQueryTestSuite/udf_pgSQL_udf_aggregates_part1_sql___Regular_Python_UDF/

```
Expected "700000000000[6] 1", but got "700000000000[5] 1" Result did not match for query #33&#010;SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS long), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3))&#010;FROM (VALUES (7000000000005), (7000000000007)) v(x)
```

Here;s what's going on: https://github.com/apache/spark/pull/25110#discussion_r302791930

```
scala> Seq("7000000000004.999", "7000000000006.999").toDF().selectExpr("CAST(avg(value) AS long)").show()
+--------------------------+
|CAST(avg(value) AS BIGINT)|
+--------------------------+
|             7000000000005|
+--------------------------+
```

Therefore, this PR just avoid to cast in the specific test.

This is a temp fix. We need more robust way to avoid such cases.

## How was this patch tested?

It passes with Maven in my local before/after this PR. I believe the problem seems similarly the Python or OS installed in the machine. I should test this against PR builder with `test-maven` for sure..

Closes #25128 from HyukjinKwon/SPARK-28270-2.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-12 14:33:16 +09:00
HyukjinKwon a5c88ecfce [SPARK-28321][SQL] 0-args Java UDF should not be called only once
## What changes were proposed in this pull request?

0-args Java UDF alone calls the function even before making it as an expression.
It causes that the function always returns the same value and the function is called at driver side.
Seems like a mistake.

## How was this patch tested?

Unit test was added

Closes #25108 from HyukjinKwon/SPARK-28321.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-12 12:44:18 +08:00
Ryan Blue 507b7457f4 [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.
## What changes were proposed in this pull request?

Implement `ALTER TABLE` for v2 tables:
* Add `AlterTable` logical plan and `AlterTableExec` physical plan
* Convert `ALTER TABLE` parsed plans to `AlterTable` when a v2 catalog is responsible for an identifier
* Validate that columns to alter exist in analyzer checks
* Fix nested type handling in `CatalogV2Util`

## How was this patch tested?

* Add extensive tests in `DataSourceV2SQLSuite`

Closes #24937 from rdblue/SPARK-28139-add-v2-alter-table.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Ryan Blue <rdblue@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-12 11:59:36 +08:00
Yuming Wang 9eca58ed3e [SPARK-28334][SQL][TEST] Port select.sql
## What changes were proposed in this pull request?

This PR is to port select.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/select.out

When porting the test cases, found four PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28010](https://issues.apache.org/jira/browse/SPARK-28010): Support ORDER BY ... USING syntax
[SPARK-28329](https://issues.apache.org/jira/browse/SPARK-28329): Support SELECT INTO syntax
[SPARK-28330](https://issues.apache.org/jira/browse/SPARK-28330): Enhance query limit
[SPARK-28296](https://issues.apache.org/jira/browse/SPARK-28296): Improved VALUES support

Also, found one inconsistent behavior:
[SPARK-28333](https://issues.apache.org/jira/browse/SPARK-28333): `NULLS FIRST` for `DESC` and `NULLS LAST` for `ASC`

## How was this patch tested?

N/A

Closes #25096 from wangyum/SPARK-28334.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-11 13:54:15 -07:00
Maxim Gekk d1ef6be4c3 [SPARK-26978][SQL][FOLLOWUP] Initialize date-time constants by foldable expressions
## What changes were proposed in this pull request?

Reverted initialization of date-time constants in `DateTimeUtils` introduced by #23878. As a comment in [Delta repo](https://github.com/delta-io/delta) states, the compiler can do additional optimizations if values can be calculated at compile time: https://github.com/delta-io/delta/blob/master/src/main/scala/org/apache/spark/sql/delta/util/DateTimeUtils.scala#L63-L75

## How was this patch tested?

This was tested by existing test suites.

Closes #25116 from MaxGekk/datetime-consts-init.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-07-11 17:48:58 +02:00
Jacek Laskowski e83583ef70 [MINOR][SQL] Clean up ObjectProducerExec operators
## What changes were proposed in this pull request?

Cleaned up (removed) code duplication in `ObjectProducerExec` operators so they use the trait's methods.

## How was this patch tested?

Local build. Waiting for Jenkins.

Closes #25065 from jaceklaskowski/ObjectProducerExec-operators-cleanup.

Authored-by: Jacek Laskowski <jacek@japila.pl>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-11 09:09:09 -05:00
Robert (Bobby) Evans 8dff711ce7 [SPARK-28213][SQL] Replace ColumnarBatchScan with equivilant from Columnar
## What changes were proposed in this pull request?

This is a second part of the https://issues.apache.org/jira/browse/SPARK-27396 and a follow on to #24795

## How was this patch tested?

I did some manual tests and ran/updated the automated tests

I did some simple performance tests on a single node to try to verify that there is no performance impact, and I was not able to measure anything beyond noise.

Closes #25008 from revans2/columnar-remove-batch-scan.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-07-11 09:03:30 -05:00
HyukjinKwon 19bcce1533 [SPARK-28270][SQL][FOLLOW-UP] Explicitly cast into int/long/decimal in udf-aggregates_part1.sql to avoid Python float limitation
## What changes were proposed in this pull request?

The tests added at https://github.com/apache/spark/pull/25069 seem flaky in some environments. See https://github.com/apache/spark/pull/25069#issuecomment-510338469

Python's string representation of floats can make the tests flaky. See https://docs.python.org/3/tutorial/floatingpoint.html.

I think it's just better to explicitly cast everywhere udf returns a float (or a double) to stay safe. (note that we're not targeting the Python <> Scala value conversions - there are inevitable differences between Python and Scala; therefore, other languages' UDFs cannot guarantee the same results between Python and Scala).

This PR proposes to cast cases to long, integer and decimal explicitly to make the test cases robust.

<details><summary>Diff comparing to 'pgSQL/aggregates_part1.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out
index 51ca1d55869..734634b7388 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out
 -3,23 +3,23

 -- !query 0
-SELECT avg(four) AS avg_1 FROM onek
+SELECT CAST(avg(udf(four)) AS decimal(10,3)) AS avg_1 FROM onek
 -- !query 0 schema
-struct<avg_1:double>
+struct<avg_1:decimal(10,3)>
 -- !query 0 output
 1.5

 -- !query 1
-SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100
+SELECT CAST(udf(avg(a)) AS decimal(10,3)) AS avg_32 FROM aggtest WHERE a < 100
 -- !query 1 schema
-struct<avg_32:double>
+struct<avg_32:decimal(10,3)>
 -- !query 1 output
-32.666666666666664
+32.667

 -- !query 2
-select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest
+select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest
 -- !query 2 schema
 struct<avg_107_943:decimal(10,3)>
 -- !query 2 output
 -27,39 +27,39  struct<avg_107_943:decimal(10,3)>

 -- !query 3
-SELECT sum(four) AS sum_1500 FROM onek
+SELECT CAST(sum(udf(four)) AS int) AS sum_1500 FROM onek
 -- !query 3 schema
-struct<sum_1500:bigint>
+struct<sum_1500:int>
 -- !query 3 output
 1500

 -- !query 4
-SELECT sum(a) AS sum_198 FROM aggtest
+SELECT udf(sum(a)) AS sum_198 FROM aggtest
 -- !query 4 schema
-struct<sum_198:bigint>
+struct<sum_198:string>
 -- !query 4 output
 198

 -- !query 5
-SELECT sum(b) AS avg_431_773 FROM aggtest
+SELECT CAST(udf(udf(sum(b))) AS decimal(10,3)) AS avg_431_773 FROM aggtest
 -- !query 5 schema
-struct<avg_431_773:double>
+struct<avg_431_773:decimal(10,3)>
 -- !query 5 output
-431.77260909229517
+431.773

 -- !query 6
-SELECT max(four) AS max_3 FROM onek
+SELECT udf(max(four)) AS max_3 FROM onek
 -- !query 6 schema
-struct<max_3:int>
+struct<max_3:string>
 -- !query 6 output
 3

 -- !query 7
-SELECT max(a) AS max_100 FROM aggtest
+SELECT max(CAST(udf(a) AS int)) AS max_100 FROM aggtest
 -- !query 7 schema
 struct<max_100:int>
 -- !query 7 output
 -67,245 +67,246  struct<max_100:int>

 -- !query 8
-SELECT max(aggtest.b) AS max_324_78 FROM aggtest
+SELECT CAST(udf(udf(max(aggtest.b))) AS decimal(10,3)) AS max_324_78 FROM aggtest
 -- !query 8 schema
-struct<max_324_78:float>
+struct<max_324_78:decimal(10,3)>
 -- !query 8 output
 324.78

 -- !query 9
-SELECT stddev_pop(b) FROM aggtest
+SELECT CAST(stddev_pop(udf(b)) AS decimal(10,3)) FROM aggtest
 -- !query 9 schema
-struct<stddev_pop(CAST(b AS DOUBLE)):double>
+struct<CAST(stddev_pop(CAST(udf(b) AS DOUBLE)) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 9 output
-131.10703231895047
+131.107

 -- !query 10
-SELECT stddev_samp(b) FROM aggtest
+SELECT CAST(udf(stddev_samp(b)) AS decimal(10,3)) FROM aggtest
 -- !query 10 schema
-struct<stddev_samp(CAST(b AS DOUBLE)):double>
+struct<CAST(udf(stddev_samp(cast(b as double))) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 10 output
-151.38936080399804
+151.389

 -- !query 11
-SELECT var_pop(b) FROM aggtest
+SELECT CAST(var_pop(udf(b)) AS decimal(10,3)) FROM aggtest
 -- !query 11 schema
-struct<var_pop(CAST(b AS DOUBLE)):double>
+struct<CAST(var_pop(CAST(udf(b) AS DOUBLE)) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 11 output
-17189.053923482323
+17189.054

 -- !query 12
-SELECT var_samp(b) FROM aggtest
+SELECT CAST(udf(var_samp(b)) AS decimal(10,3)) FROM aggtest
 -- !query 12 schema
-struct<var_samp(CAST(b AS DOUBLE)):double>
+struct<CAST(udf(var_samp(cast(b as double))) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 12 output
-22918.738564643096
+22918.739

 -- !query 13
-SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT CAST(udf(stddev_pop(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest
 -- !query 13 schema
-struct<stddev_pop(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(udf(stddev_pop(cast(cast(b as decimal(38,0)) as double))) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 13 output
-131.18117242958306
+131.181

 -- !query 14
-SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT CAST(stddev_samp(CAST(udf(b) AS Decimal(38,0))) AS decimal(10,3)) FROM aggtest
 -- !query 14 schema
-struct<stddev_samp(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(stddev_samp(CAST(CAST(udf(b) AS DECIMAL(38,0)) AS DOUBLE)) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 14 output
-151.47497042966097
+151.475

 -- !query 15
-SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT CAST(udf(var_pop(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest
 -- !query 15 schema
-struct<var_pop(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(udf(var_pop(cast(cast(b as decimal(38,0)) as double))) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 15 output
 17208.5

 -- !query 16
-SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT CAST(var_samp(udf(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest
 -- !query 16 schema
-struct<var_samp(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(var_samp(CAST(udf(cast(b as decimal(38,0))) AS DOUBLE)) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 16 output
-22944.666666666668
+22944.667

 -- !query 17
-SELECT var_pop(1.0), var_samp(2.0)
+SELECT CAST(udf(var_pop(1.0)) AS int), var_samp(udf(2.0))
 -- !query 17 schema
-struct<var_pop(CAST(1.0 AS DOUBLE)):double,var_samp(CAST(2.0 AS DOUBLE)):double>
+struct<CAST(udf(var_pop(cast(1.0 as double))) AS INT):int,var_samp(CAST(udf(2.0) AS DOUBLE)):double>
 -- !query 17 output
-0.0    NaN
+0      NaN

 -- !query 18
-SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0)))
+SELECT CAST(stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))) AS int), stddev_samp(CAST(udf(4.0) AS Decimal(38,0)))
 -- !query 18 schema
-struct<stddev_pop(CAST(CAST(3.0 AS DECIMAL(38,0)) AS DOUBLE)):double,stddev_samp(CAST(CAST(4.0 AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<CAST(stddev_pop(CAST(udf(cast(3.0 as decimal(38,0))) AS DOUBLE)) AS INT):int,stddev_samp(CAST(CAST(udf(4.0) AS DECIMAL(38,0)) AS DOUBLE)):double>
 -- !query 18 output
-0.0    NaN
+0      NaN

 -- !query 19
-select sum(CAST(null AS int)) from range(1,4)
+select sum(udf(CAST(null AS int))) from range(1,4)
 -- !query 19 schema
-struct<sum(CAST(NULL AS INT)):bigint>
+struct<sum(CAST(udf(cast(null as int)) AS DOUBLE)):double>
 -- !query 19 output
 NULL

 -- !query 20
-select sum(CAST(null AS long)) from range(1,4)
+select sum(udf(CAST(null AS long))) from range(1,4)
 -- !query 20 schema
-struct<sum(CAST(NULL AS BIGINT)):bigint>
+struct<sum(CAST(udf(cast(null as bigint)) AS DOUBLE)):double>
 -- !query 20 output
 NULL

 -- !query 21
-select sum(CAST(null AS Decimal(38,0))) from range(1,4)
+select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4)
 -- !query 21 schema
-struct<sum(CAST(NULL AS DECIMAL(38,0))):decimal(38,0)>
+struct<sum(CAST(udf(cast(null as decimal(38,0))) AS DOUBLE)):double>
 -- !query 21 output
 NULL

 -- !query 22
-select sum(CAST(null AS DOUBLE)) from range(1,4)
+select sum(udf(CAST(null AS DOUBLE))) from range(1,4)
 -- !query 22 schema
-struct<sum(CAST(NULL AS DOUBLE)):double>
+struct<sum(CAST(udf(cast(null as double)) AS DOUBLE)):double>
 -- !query 22 output
 NULL

 -- !query 23
-select avg(CAST(null AS int)) from range(1,4)
+select avg(udf(CAST(null AS int))) from range(1,4)
 -- !query 23 schema
-struct<avg(CAST(NULL AS INT)):double>
+struct<avg(CAST(udf(cast(null as int)) AS DOUBLE)):double>
 -- !query 23 output
 NULL

 -- !query 24
-select avg(CAST(null AS long)) from range(1,4)
+select avg(udf(CAST(null AS long))) from range(1,4)
 -- !query 24 schema
-struct<avg(CAST(NULL AS BIGINT)):double>
+struct<avg(CAST(udf(cast(null as bigint)) AS DOUBLE)):double>
 -- !query 24 output
 NULL

 -- !query 25
-select avg(CAST(null AS Decimal(38,0))) from range(1,4)
+select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4)
 -- !query 25 schema
-struct<avg(CAST(NULL AS DECIMAL(38,0))):decimal(38,4)>
+struct<avg(CAST(udf(cast(null as decimal(38,0))) AS DOUBLE)):double>
 -- !query 25 output
 NULL

 -- !query 26
-select avg(CAST(null AS DOUBLE)) from range(1,4)
+select avg(udf(CAST(null AS DOUBLE))) from range(1,4)
 -- !query 26 schema
-struct<avg(CAST(NULL AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(null as double)) AS DOUBLE)):double>
 -- !query 26 output
 NULL

 -- !query 27
-select sum(CAST('NaN' AS DOUBLE)) from range(1,4)
+select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4)
 -- !query 27 schema
-struct<sum(CAST(NaN AS DOUBLE)):double>
+struct<sum(CAST(udf(NaN) AS DOUBLE)):double>
 -- !query 27 output
 NaN

 -- !query 28
-select avg(CAST('NaN' AS DOUBLE)) from range(1,4)
+select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4)
 -- !query 28 schema
-struct<avg(CAST(NaN AS DOUBLE)):double>
+struct<avg(CAST(udf(NaN) AS DOUBLE)):double>
 -- !query 28 output
 NaN

 -- !query 30
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('Infinity'), ('1')) v(x)
 -- !query 30 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(x) AS DOUBLE)):double,var_pop(CAST(udf(x) AS DOUBLE)):double>
 -- !query 30 output
 Infinity       NaN

 -- !query 31
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('Infinity'), ('Infinity')) v(x)
 -- !query 31 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(x) AS DOUBLE)):double,var_pop(CAST(udf(x) AS DOUBLE)):double>
 -- !query 31 output
 Infinity       NaN

 -- !query 32
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('-Infinity'), ('Infinity')) v(x)
 -- !query 32 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(x) AS DOUBLE)):double,var_pop(CAST(udf(x) AS DOUBLE)):double>
 -- !query 32 output
 NaN    NaN

 -- !query 33
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS int), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3))
 FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x)
 -- !query 33 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<CAST(avg(CAST(udf(cast(x as double)) AS DOUBLE)) AS INT):int,CAST(udf(var_pop(cast(x as double))) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 33 output
-1.00000005E8   2.5
+100000005      2.5

 -- !query 34
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS long), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3))
 FROM (VALUES (7000000000005), (7000000000007)) v(x)
 -- !query 34 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<CAST(avg(CAST(udf(cast(x as double)) AS DOUBLE)) AS BIGINT):bigint,CAST(udf(var_pop(cast(x as double))) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 34 output
-7.000000000006E12      1.0
+7000000000006  1

 -- !query 35
-SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest
+SELECT CAST(udf(covar_pop(b, udf(a))) AS decimal(10,3)), CAST(covar_samp(udf(b), a) as decimal(10,3)) FROM aggtest
 -- !query 35 schema
-struct<covar_pop(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double,covar_samp(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double>
+struct<CAST(udf(covar_pop(cast(b as double), cast(udf(a) as double))) AS DECIMAL(10,3)):decimal(10,3),CAST(covar_samp(CAST(udf(b) AS DOUBLE), CAST(a AS DOUBLE)) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 35 output
-653.6289553875104      871.5052738500139
+653.629        871.505

 -- !query 36
-SELECT corr(b, a) FROM aggtest
+SELECT CAST(corr(b, udf(a)) AS decimal(10,3)) FROM aggtest
 -- !query 36 schema
-struct<corr(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double>
+struct<CAST(corr(CAST(b AS DOUBLE), CAST(udf(a) AS DOUBLE)) AS DECIMAL(10,3)):decimal(10,3)>
 -- !query 36 output
-0.1396345165178734
+0.14

 -- !query 37
-SELECT count(four) AS cnt_1000 FROM onek
+SELECT count(udf(four)) AS cnt_1000 FROM onek
 -- !query 37 schema
 struct<cnt_1000:bigint>
 -- !query 37 output
 -313,18 +314,18  struct<cnt_1000:bigint>

 -- !query 38
-SELECT count(DISTINCT four) AS cnt_4 FROM onek
+SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek
 -- !query 38 schema
-struct<cnt_4:bigint>
+struct<cnt_4:string>
 -- !query 38 output
 4

 -- !query 39
-select ten, count(*), sum(four) from onek
+select ten, udf(count(*)), CAST(sum(udf(four)) AS int) from onek
 group by ten order by ten
 -- !query 39 schema
-struct<ten:int,count(1):bigint,sum(four):bigint>
+struct<ten:int,udf(count(1)):string,CAST(sum(CAST(udf(four) AS DOUBLE)) AS INT):int>
 -- !query 39 output
 0      100     100
 1      100     200
 -339,10 +340,10  struct<ten:int,count(1):bigint,sum(four):bigint>

 -- !query 40
-select ten, count(four), sum(DISTINCT four) from onek
+select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek
 group by ten order by ten
 -- !query 40 schema
-struct<ten:int,count(four):bigint,sum(DISTINCT four):bigint>
+struct<ten:int,count(udf(four)):bigint,udf(sum(distinct cast(four as bigint))):string>
 -- !query 40 output
 0      100     2
 1      100     4
 -357,11 +358,11  struct<ten:int,count(four):bigint,sum(DISTINCT four):bigint>

 -- !query 41
-select ten, sum(distinct four) from onek a
+select ten, udf(sum(distinct four)) from onek a
 group by ten
-having exists (select 1 from onek b where sum(distinct a.four) = b.four)
+having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four)
 -- !query 41 schema
-struct<ten:int,sum(DISTINCT four):bigint>
+struct<ten:int,udf(sum(distinct cast(four as bigint))):string>
 -- !query 41 output
 0      2
 2      2
 -374,23 +375,23  struct<ten:int,sum(DISTINCT four):bigint>
 select ten, sum(distinct four) from onek a
 group by ten
 having exists (select 1 from onek b
-               where sum(distinct a.four + b.four) = b.four)
+               where sum(distinct a.four + b.four) = udf(b.four))
 -- !query 42 schema
 struct<>
 -- !query 42 output
 org.apache.spark.sql.AnalysisException

 Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(b.`four` AS BIGINT))]
+Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(udf(four) AS BIGINT))]
 Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))];

 -- !query 43
 select
-  (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))
+  (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))))
 from tenk1 o
 -- !query 43 schema
 struct<>
 -- !query 43 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 63
+cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 67
```

</p>
</details>

## How was this patch tested?

Manually tested in local.

Also, with JDK 11:

```
Using /.../jdk-11.0.3.jdk/Contents/Home as default JAVA_HOME.
Note, this will be overridden by -java-home if it is set.
[info] Loading project definition from /.../spark/project
[info] Updating {file:/.../spark/project/}spark-build...
...
[info] SQLQueryTestSuite:
...
[info] - udf/pgSQL/udf-aggregates_part1.sql - Scala UDF (17 seconds, 228 milliseconds)
[info] - udf/pgSQL/udf-aggregates_part1.sql - Regular Python UDF (36 seconds, 170 milliseconds)
[info] - udf/pgSQL/udf-aggregates_part1.sql - Scalar Pandas UDF (41 seconds, 132 milliseconds)
...
```

Closes #25110 from HyukjinKwon/SPARK-28270-1.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 20:52:54 +09:00
Yesheng Ma 7021588ba8 [SPARK-28306][SQL] Make NormalizeFloatingNumbers rule idempotent
## What changes were proposed in this pull request?
The optimizer rule `NormalizeFloatingNumbers` is not idempotent. It will generate multiple `NormalizeNaNAndZero` and `ArrayTransform` expression nodes for multiple runs. This patch fixed this non-idempotence by adding a marking tag above normalized expressions. It also adds missing UTs for `NormalizeFloatingNumbers`.

## How was this patch tested?
New UTs.

Closes #25080 from yeshengm/spark-28306.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-11 10:22:00 +08:00
HyukjinKwon 019762816a [SPARK-28342][SQL][TESTS] Replace REL_12_BETA1 to REL_12_BETA2 in PostgresSQL SQL tests
## What changes were proposed in this pull request?

This PR proposes to replace `REL_12_BETA1` to `REL_12_BETA2` which is latest.

## How was this patch tested?

Manually checked each link and checked via `git grep -r REL_12_BETA1` as well.

Closes #25105 from HyukjinKwon/SPARK-28342.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 19:02:27 -07:00
Carson Wang 3f375c850b [SPARK-28339][SQL] Rename Spark SQL adaptive execution configuration name
## What changes were proposed in this pull request?
The new adaptive execution framework introduced configuration `spark.sql.runtime.reoptimization.enabled`. We now rename it back to `spark.sql.adaptive.enabled` as the umbrella configuration for adaptive execution.

## How was this patch tested?
Existing tests.

Closes #25102 from carsonwang/renameAE.

Authored-by: Carson Wang <carson.wang@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-11 09:17:45 +08:00
HyukjinKwon 92e051caf9 [SPARK-28270][SQL][PYTHON] Convert and port 'pgSQL/aggregates_part1.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `pgSQL/aggregates_part1.sql'` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

This PR also contains two minor fixes:

1. Change name of Scala UDF from `UDF:name(...)` to `name(...)` to be consistent with Python'

2. Fix Scala UDF at `IntegratedUDFTestUtils.scala ` to handle `null` in strings.

<details><summary>Diff comparing to 'pgSQL/aggregates_part1.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out
index 51ca1d55869..124fdd6416e 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out
 -3,7 +3,7

 -- !query 0
-SELECT avg(four) AS avg_1 FROM onek
+SELECT avg(udf(four)) AS avg_1 FROM onek
 -- !query 0 schema
 struct<avg_1:double>
 -- !query 0 output
 -11,15 +11,15  struct<avg_1:double>

 -- !query 1
-SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100
+SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100
 -- !query 1 schema
-struct<avg_32:double>
+struct<avg_32:string>
 -- !query 1 output
 32.666666666666664

 -- !query 2
-select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest
+select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest
 -- !query 2 schema
 struct<avg_107_943:decimal(10,3)>
 -- !query 2 output
 -27,285 +27,286  struct<avg_107_943:decimal(10,3)>

 -- !query 3
-SELECT sum(four) AS sum_1500 FROM onek
+SELECT sum(udf(four)) AS sum_1500 FROM onek
 -- !query 3 schema
-struct<sum_1500:bigint>
+struct<sum_1500:double>
 -- !query 3 output
-1500
+1500.0

 -- !query 4
-SELECT sum(a) AS sum_198 FROM aggtest
+SELECT udf(sum(a)) AS sum_198 FROM aggtest
 -- !query 4 schema
-struct<sum_198:bigint>
+struct<sum_198:string>
 -- !query 4 output
 198

 -- !query 5
-SELECT sum(b) AS avg_431_773 FROM aggtest
+SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest
 -- !query 5 schema
-struct<avg_431_773:double>
+struct<avg_431_773:string>
 -- !query 5 output
 431.77260909229517

 -- !query 6
-SELECT max(four) AS max_3 FROM onek
+SELECT udf(max(four)) AS max_3 FROM onek
 -- !query 6 schema
-struct<max_3:int>
+struct<max_3:string>
 -- !query 6 output
 3

 -- !query 7
-SELECT max(a) AS max_100 FROM aggtest
+SELECT max(udf(a)) AS max_100 FROM aggtest
 -- !query 7 schema
-struct<max_100:int>
+struct<max_100:string>
 -- !query 7 output
-100
+56

 -- !query 8
-SELECT max(aggtest.b) AS max_324_78 FROM aggtest
+SELECT CAST(udf(udf(max(aggtest.b))) AS int) AS max_324_78 FROM aggtest
 -- !query 8 schema
-struct<max_324_78:float>
+struct<max_324_78:int>
 -- !query 8 output
-324.78
+324

 -- !query 9
-SELECT stddev_pop(b) FROM aggtest
+SELECT CAST(stddev_pop(udf(b)) AS int) FROM aggtest
 -- !query 9 schema
-struct<stddev_pop(CAST(b AS DOUBLE)):double>
+struct<CAST(stddev_pop(CAST(udf(b) AS DOUBLE)) AS INT):int>
 -- !query 9 output
-131.10703231895047
+131

 -- !query 10
-SELECT stddev_samp(b) FROM aggtest
+SELECT udf(stddev_samp(b)) FROM aggtest
 -- !query 10 schema
-struct<stddev_samp(CAST(b AS DOUBLE)):double>
+struct<udf(stddev_samp(cast(b as double))):string>
 -- !query 10 output
 151.38936080399804

 -- !query 11
-SELECT var_pop(b) FROM aggtest
+SELECT CAST(var_pop(udf(b)) as int) FROM aggtest
 -- !query 11 schema
-struct<var_pop(CAST(b AS DOUBLE)):double>
+struct<CAST(var_pop(CAST(udf(b) AS DOUBLE)) AS INT):int>
 -- !query 11 output
-17189.053923482323
+17189

 -- !query 12
-SELECT var_samp(b) FROM aggtest
+SELECT udf(var_samp(b)) FROM aggtest
 -- !query 12 schema
-struct<var_samp(CAST(b AS DOUBLE)):double>
+struct<udf(var_samp(cast(b as double))):string>
 -- !query 12 output
 22918.738564643096

 -- !query 13
-SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest
 -- !query 13 schema
-struct<stddev_pop(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<udf(stddev_pop(cast(cast(b as decimal(38,0)) as double))):string>
 -- !query 13 output
 131.18117242958306

 -- !query 14
-SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest
 -- !query 14 schema
-struct<stddev_samp(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<stddev_samp(CAST(CAST(udf(b) AS DECIMAL(38,0)) AS DOUBLE)):double>
 -- !query 14 output
 151.47497042966097

 -- !query 15
-SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest
 -- !query 15 schema
-struct<var_pop(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<udf(var_pop(cast(cast(b as decimal(38,0)) as double))):string>
 -- !query 15 output
 17208.5

 -- !query 16
-SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest
+SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest
 -- !query 16 schema
-struct<var_samp(CAST(CAST(b AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<var_samp(CAST(udf(cast(b as decimal(38,0))) AS DOUBLE)):double>
 -- !query 16 output
 22944.666666666668

 -- !query 17
-SELECT var_pop(1.0), var_samp(2.0)
+SELECT udf(var_pop(1.0)), var_samp(udf(2.0))
 -- !query 17 schema
-struct<var_pop(CAST(1.0 AS DOUBLE)):double,var_samp(CAST(2.0 AS DOUBLE)):double>
+struct<udf(var_pop(cast(1.0 as double))):string,var_samp(CAST(udf(2.0) AS DOUBLE)):double>
 -- !query 17 output
 0.0    NaN

 -- !query 18
-SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0)))
+SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0)))
 -- !query 18 schema
-struct<stddev_pop(CAST(CAST(3.0 AS DECIMAL(38,0)) AS DOUBLE)):double,stddev_samp(CAST(CAST(4.0 AS DECIMAL(38,0)) AS DOUBLE)):double>
+struct<stddev_pop(CAST(udf(cast(3.0 as decimal(38,0))) AS DOUBLE)):double,stddev_samp(CAST(CAST(udf(4.0) AS DECIMAL(38,0)) AS DOUBLE)):double>
 -- !query 18 output
 0.0    NaN

 -- !query 19
-select sum(CAST(null AS int)) from range(1,4)
+select sum(udf(CAST(null AS int))) from range(1,4)
 -- !query 19 schema
-struct<sum(CAST(NULL AS INT)):bigint>
+struct<sum(CAST(udf(cast(null as int)) AS DOUBLE)):double>
 -- !query 19 output
 NULL

 -- !query 20
-select sum(CAST(null AS long)) from range(1,4)
+select sum(udf(CAST(null AS long))) from range(1,4)
 -- !query 20 schema
-struct<sum(CAST(NULL AS BIGINT)):bigint>
+struct<sum(CAST(udf(cast(null as bigint)) AS DOUBLE)):double>
 -- !query 20 output
 NULL

 -- !query 21
-select sum(CAST(null AS Decimal(38,0))) from range(1,4)
+select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4)
 -- !query 21 schema
-struct<sum(CAST(NULL AS DECIMAL(38,0))):decimal(38,0)>
+struct<sum(CAST(udf(cast(null as decimal(38,0))) AS DOUBLE)):double>
 -- !query 21 output
 NULL

 -- !query 22
-select sum(CAST(null AS DOUBLE)) from range(1,4)
+select sum(udf(CAST(null AS DOUBLE))) from range(1,4)
 -- !query 22 schema
-struct<sum(CAST(NULL AS DOUBLE)):double>
+struct<sum(CAST(udf(cast(null as double)) AS DOUBLE)):double>
 -- !query 22 output
 NULL

 -- !query 23
-select avg(CAST(null AS int)) from range(1,4)
+select avg(udf(CAST(null AS int))) from range(1,4)
 -- !query 23 schema
-struct<avg(CAST(NULL AS INT)):double>
+struct<avg(CAST(udf(cast(null as int)) AS DOUBLE)):double>
 -- !query 23 output
 NULL

 -- !query 24
-select avg(CAST(null AS long)) from range(1,4)
+select avg(udf(CAST(null AS long))) from range(1,4)
 -- !query 24 schema
-struct<avg(CAST(NULL AS BIGINT)):double>
+struct<avg(CAST(udf(cast(null as bigint)) AS DOUBLE)):double>
 -- !query 24 output
 NULL

 -- !query 25
-select avg(CAST(null AS Decimal(38,0))) from range(1,4)
+select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4)
 -- !query 25 schema
-struct<avg(CAST(NULL AS DECIMAL(38,0))):decimal(38,4)>
+struct<avg(CAST(udf(cast(null as decimal(38,0))) AS DOUBLE)):double>
 -- !query 25 output
 NULL

 -- !query 26
-select avg(CAST(null AS DOUBLE)) from range(1,4)
+select avg(udf(CAST(null AS DOUBLE))) from range(1,4)
 -- !query 26 schema
-struct<avg(CAST(NULL AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(null as double)) AS DOUBLE)):double>
 -- !query 26 output
 NULL

 -- !query 27
-select sum(CAST('NaN' AS DOUBLE)) from range(1,4)
+select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4)
 -- !query 27 schema
-struct<sum(CAST(NaN AS DOUBLE)):double>
+struct<sum(CAST(udf(NaN) AS DOUBLE)):double>
 -- !query 27 output
 NaN

 -- !query 28
-select avg(CAST('NaN' AS DOUBLE)) from range(1,4)
+select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4)
 -- !query 28 schema
-struct<avg(CAST(NaN AS DOUBLE)):double>
+struct<avg(CAST(udf(NaN) AS DOUBLE)):double>
 -- !query 28 output
 NaN

 -- !query 29
 SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
-FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('Infinity' AS DOUBLE))) v(x)
+FROM (VALUES (CAST(udf('1') AS DOUBLE)), (CAST(udf('Infinity') AS DOUBLE))) v(x)
 -- !query 29 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<>
 -- !query 29 output
-Infinity       NaN
+org.apache.spark.sql.AnalysisException
+cannot evaluate expression CAST(udf(1) AS DOUBLE) in inline table definition; line 2 pos 14

 -- !query 30
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('Infinity'), ('1')) v(x)
 -- !query 30 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(x) AS DOUBLE)):double,var_pop(CAST(udf(x) AS DOUBLE)):double>
 -- !query 30 output
 Infinity       NaN

 -- !query 31
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('Infinity'), ('Infinity')) v(x)
 -- !query 31 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(x) AS DOUBLE)):double,var_pop(CAST(udf(x) AS DOUBLE)):double>
 -- !query 31 output
 Infinity       NaN

 -- !query 32
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE))
 FROM (VALUES ('-Infinity'), ('Infinity')) v(x)
 -- !query 32 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(x) AS DOUBLE)):double,var_pop(CAST(udf(x) AS DOUBLE)):double>
 -- !query 32 output
 NaN    NaN

 -- !query 33
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE)))
 FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x)
 -- !query 33 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(x as double)) AS DOUBLE)):double,udf(var_pop(cast(x as double))):string>
 -- !query 33 output
 1.00000005E8   2.5

 -- !query 34
-SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
+SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE)))
 FROM (VALUES (7000000000005), (7000000000007)) v(x)
 -- !query 34 schema
-struct<avg(CAST(x AS DOUBLE)):double,var_pop(CAST(x AS DOUBLE)):double>
+struct<avg(CAST(udf(cast(x as double)) AS DOUBLE)):double,udf(var_pop(cast(x as double))):string>
 -- !query 34 output
 7.000000000006E12      1.0

 -- !query 35
-SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest
+SELECT CAST(udf(covar_pop(b, udf(a))) AS int), CAST(covar_samp(udf(b), a) as int) FROM aggtest
 -- !query 35 schema
-struct<covar_pop(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double,covar_samp(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double>
+struct<CAST(udf(covar_pop(cast(b as double), cast(udf(a) as double))) AS INT):int,CAST(covar_samp(CAST(udf(b) AS DOUBLE), CAST(a AS DOUBLE)) AS INT):int>
 -- !query 35 output
-653.6289553875104      871.5052738500139
+653    871

 -- !query 36
-SELECT corr(b, a) FROM aggtest
+SELECT corr(b, udf(a)) FROM aggtest
 -- !query 36 schema
-struct<corr(CAST(b AS DOUBLE), CAST(a AS DOUBLE)):double>
+struct<corr(CAST(b AS DOUBLE), CAST(udf(a) AS DOUBLE)):double>
 -- !query 36 output
 0.1396345165178734

 -- !query 37
-SELECT count(four) AS cnt_1000 FROM onek
+SELECT count(udf(four)) AS cnt_1000 FROM onek
 -- !query 37 schema
 struct<cnt_1000:bigint>
 -- !query 37 output
 -313,36 +314,36  struct<cnt_1000:bigint>

 -- !query 38
-SELECT count(DISTINCT four) AS cnt_4 FROM onek
+SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek
 -- !query 38 schema
-struct<cnt_4:bigint>
+struct<cnt_4:string>
 -- !query 38 output
 4

 -- !query 39
-select ten, count(*), sum(four) from onek
+select ten, udf(count(*)), sum(udf(four)) from onek
 group by ten order by ten
 -- !query 39 schema
-struct<ten:int,count(1):bigint,sum(four):bigint>
+struct<ten:int,udf(count(1)):string,sum(CAST(udf(four) AS DOUBLE)):double>
 -- !query 39 output
-0      100     100
-1      100     200
-2      100     100
-3      100     200
-4      100     100
-5      100     200
-6      100     100
-7      100     200
-8      100     100
-9      100     200
+0      100     100.0
+1      100     200.0
+2      100     100.0
+3      100     200.0
+4      100     100.0
+5      100     200.0
+6      100     100.0
+7      100     200.0
+8      100     100.0
+9      100     200.0

 -- !query 40
-select ten, count(four), sum(DISTINCT four) from onek
+select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek
 group by ten order by ten
 -- !query 40 schema
-struct<ten:int,count(four):bigint,sum(DISTINCT four):bigint>
+struct<ten:int,count(udf(four)):bigint,udf(sum(distinct cast(four as bigint))):string>
 -- !query 40 output
 0      100     2
 1      100     4
 -357,11 +358,11  struct<ten:int,count(four):bigint,sum(DISTINCT four):bigint>

 -- !query 41
-select ten, sum(distinct four) from onek a
+select ten, udf(sum(distinct four)) from onek a
 group by ten
-having exists (select 1 from onek b where sum(distinct a.four) = b.four)
+having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four)
 -- !query 41 schema
-struct<ten:int,sum(DISTINCT four):bigint>
+struct<ten:int,udf(sum(distinct cast(four as bigint))):string>
 -- !query 41 output
 0      2
 2      2
 -374,23 +375,23  struct<ten:int,sum(DISTINCT four):bigint>
 select ten, sum(distinct four) from onek a
 group by ten
 having exists (select 1 from onek b
-               where sum(distinct a.four + b.four) = b.four)
+               where sum(distinct a.four + b.four) = udf(b.four))
 -- !query 42 schema
 struct<>
 -- !query 42 output
 org.apache.spark.sql.AnalysisException

 Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(b.`four` AS BIGINT))]
+Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(udf(four) AS BIGINT))]
 Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))];

 -- !query 43
 select
-  (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))
+  (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))))
 from tenk1 o
 -- !query 43 schema
 struct<>
 -- !query 43 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 63
+cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 67
```

</p>
</details>

Note that, currently, `IntegratedUDFTestUtils.scala`'s UDFs only return strings. There are some differences between those UDFs (Scala, Pandas and Python):

  - Python's string representation of floats can make the tests flaky. (See https://docs.python.org/3/tutorial/floatingpoint.html). To work around this, I had to `CAST(... as int)`.
  - There are string representation differences between `Inf` `-Inf` <> `Infinity` `-Infinity` and  `nan` <> `NaN`
  - Maybe we should add other type versions of UDFs if this makes adding tests difficult.

Note that one issue found - [SPARK-28291](https://issues.apache.org/jira/browse/SPARK-28291). The test was commented for now.

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25069 from HyukjinKwon/SPARK-28270.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 10:12:23 +09:00
Maxim Gekk 653215377a [SPARK-28015][SQL] Check stringToDate() consumes entire input for the yyyy and yyyy-[m]m formats
## What changes were proposed in this pull request?

Fix `stringToDate()` for the formats `yyyy` and `yyyy-[m]m` that assumes there are no additional chars after the last components `yyyy` and `[m]m`. In the PR, I propose to check that entire input was consumed for the formats.

After the fix, the input `1999 08 01` will be invalid because it matches to the pattern `yyyy` but the strings contains additional chars ` 08 01`.

Since Spark 1.6.3 ~ 2.4.3, the behavior is the same.
```
spark-sql> SELECT CAST('1999 08 01' AS DATE);
1999-01-01
```

This PR makes it return NULL like Hive.
```
spark-sql> SELECT CAST('1999 08 01' AS DATE);
NULL
```

## How was this patch tested?

Added new checks to `DateTimeUtilsSuite` for the `1999 08 01` and `1999 08` inputs.

Closes #25097 from MaxGekk/spark-28015-invalid-date-format.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 18:12:03 -07:00
Ryan Blue ec821b4411 [SPARK-27919][SQL] Add v2 session catalog
## What changes were proposed in this pull request?

This fixes a problem where it is possible to create a v2 table using the default catalog that cannot be loaded with the session catalog. A session catalog should be used when the v1 catalog is responsible for tables with no catalog in the table identifier.

* Adds a v2 catalog implementation that delegates to the analyzer's SessionCatalog
* Uses the v2 session catalog for CTAS and CreateTable when the provider is a v2 provider and no v2 catalog is in the table identifier
* Updates catalog lookup to always provide the default if it is set for consistent behavior

## How was this patch tested?

* Adds a new test suite for the v2 session catalog that validates the TableCatalog API
* Adds test cases in PlanResolutionSuite to validate the v2 session catalog is used
* Adds test suite for LookupCatalog with a default catalog

Closes #24768 from rdblue/SPARK-27919-add-v2-session-catalog.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-11 09:10:30 +08:00
Zhu, Lipeng d26642dbbc [SPARK-28107][SQL] Support 'DAY TO (HOUR|MINUTE|SECOND)', 'HOUR TO (MINUTE|SECOND)' and 'MINUTE TO SECOND'
## What changes were proposed in this pull request?
The interval conversion behavior is same with the PostgreSQL.

https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/interval.sql#L180-L203

## How was this patch tested?
UT.

Closes #25000 from lipzhu/SPARK-28107.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 18:01:42 -07:00
Huaxin Gao 3a94fb3dd9 [SPARK-28281][SQL][PYTHON][TESTS] Convert and port 'having.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from having.sql to test UDFs following the combination guide in [SPARK-27921](url)
<details><summary>Diff comparing to 'having.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out
index d87ee52216..7cea2e5128 100644
--- a/sql/core/src/test/resources/sql-tests/results/having.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out
 -16,34 +16,34  struct<>

 -- !query 1
-SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2
+SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2
 -- !query 1 schema
-struct<k:string,sum(v):bigint>
+struct<k:string,udf(sum(cast(v as bigint))):string>
 -- !query 1 output
 one    6
 three  3

 -- !query 2
-SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2
+SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2)
 -- !query 2 schema
-struct<count(k):bigint>
+struct<udf(count(udf(k))):string>
 -- !query 2 output
 1

 -- !query 3
-SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0)
+SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0)
 -- !query 3 schema
-struct<min(v):int>
+struct<udf(min(v)):string>
 -- !query 3 output
 1

 -- !query 4
-SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1
+SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1)
 -- !query 4 schema
-struct<(a + CAST(b AS BIGINT)):bigint>
+struct<udf((a + cast(b as bigint))):string>
 -- !query 4 output
 3
 7

```

</p>
</details>

## How was this patch tested?

Tested as guided in SPARK-27921.

Closes #25093 from huaxingao/spark-28281.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 09:57:34 +09:00
Terry Kim 8d686f34fc [SPARK-28271][SQL][PYTHON][TESTS] Convert and port 'pgSQL/aggregates_part2.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `pgSQL/aggregates_part2.sql'` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff comparing to 'pgSQL/aggregates_part2.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out
index 2606d2eba7..00c06f94b5 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out
 -57,23 +57,23  true        false   true    false   true    true    true    true    true

 -- !query 3
-select min(unique1) from tenk1
+select min(udf(unique1)) from tenk1
 -- !query 3 schema
-struct<min(unique1):int>
+struct<min(udf(unique1)):string>
 -- !query 3 output
 0

 -- !query 4
-select max(unique1) from tenk1
+select udf(max(unique1)) from tenk1
 -- !query 4 schema
-struct<max(unique1):int>
+struct<udf(max(unique1)):string>
 -- !query 4 output
 9999

 -- !query 5
-select max(unique1) from tenk1 where unique1 < 42
+select max(unique1) from tenk1 where udf(unique1) < 42
 -- !query 5 schema
 struct<max(unique1):int>
 -- !query 5 output
 -81,7 +81,7  struct<max(unique1):int>

 -- !query 6
-select max(unique1) from tenk1 where unique1 > 42
+select max(unique1) from tenk1 where unique1 > udf(42)
 -- !query 6 schema
 struct<max(unique1):int>
 -- !query 6 output
 -89,7 +89,7  struct<max(unique1):int>

 -- !query 7
-select max(unique1) from tenk1 where unique1 > 42000
+select max(unique1) from tenk1 where udf(unique1) > 42000
 -- !query 7 schema
 struct<max(unique1):int>
 -- !query 7 output
 -97,7 +97,7  NULL

 -- !query 8
-select max(tenthous) from tenk1 where thousand = 33
+select max(tenthous) from tenk1 where udf(thousand) = 33
 -- !query 8 schema
 struct<max(tenthous):int>
 -- !query 8 output
 -105,7 +105,7  struct<max(tenthous):int>

 -- !query 9
-select min(tenthous) from tenk1 where thousand = 33
+select min(tenthous) from tenk1 where udf(thousand) = 33
 -- !query 9 schema
 struct<min(tenthous):int>
 -- !query 9 output
 -113,15 +113,15  struct<min(tenthous):int>

 -- !query 10
-select distinct max(unique2) from tenk1
+select distinct max(udf(unique2)) from tenk1
 -- !query 10 schema
-struct<max(unique2):int>
+struct<max(udf(unique2)):string>
 -- !query 10 output
 9999

 -- !query 11
-select max(unique2) from tenk1 order by 1
+select max(unique2) from tenk1 order by udf(1)
 -- !query 11 schema
 struct<max(unique2):int>
 -- !query 11 output
 -129,7 +129,7  struct<max(unique2):int>

 -- !query 12
-select max(unique2) from tenk1 order by max(unique2)
+select max(unique2) from tenk1 order by max(udf(unique2))
 -- !query 12 schema
 struct<max(unique2):int>
 -- !query 12 output
 -137,7 +137,7  struct<max(unique2):int>

 -- !query 13
-select max(unique2) from tenk1 order by max(unique2)+1
+select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1
 -- !query 13 schema
-struct<max(unique2):int>
+struct<udf(max(udf(unique2))):string>
 -- !query 13 output
 9999

 -- !query 14
-select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc
+select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc
 -- !query 14 schema
-struct<max_unique2:int,g:int>
+struct<max_unique2:string,udf(g):string>
 -- !query 14 output
 9999   3
 9999   2
 -155,8 +155,8  struct<max_unique2:int,g:int>

 -- !query 15
-select max(100) from tenk1
+select udf(max(100)) from tenk1
 -- !query 15 schema
-struct<max(100):int>
+struct<udf(max(100)):string>
 -- !query 15 output
 100
```

</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25086 from imback82/udf_test.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 09:45:38 +09:00
Vinod KC b598dfd5b4 [SPARK-28275][SQL][PYTHON][TESTS] Convert and port 'count.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from 'count.sql' to test UDFs

<details><summary>Diff comparing to 'count.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/count.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out
index b8a86d4c44..9476937abd 100644
--- a/sql/core/src/test/resources/sql-tests/results/count.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out
 -14,42 +14,42  struct<>

 -- !query 1
 SELECT
-  count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b))
+  udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b)))
 FROM testData
 -- !query 1 schema
-struct<count(1):bigint,count(1):bigint,count(NULL):bigint,count(a):bigint,count(b):bigint,count((a + b)):bigint,count(named_struct(a, a, b, b)):bigint>
+struct<udf(count(1)):string,udf(count(1)):string,udf(count(null)):string,udf(count(a)):string,udf(count(b)):string,udf(count((a + b))):string,udf(count(named_struct(a, a, b, b))):string>
 -- !query 1 output
 7	7	0	5	5	4	7

 -- !query 2
 SELECT
-  count(DISTINCT 1),
-  count(DISTINCT null),
-  count(DISTINCT a),
-  count(DISTINCT b),
-  count(DISTINCT (a + b)),
-  count(DISTINCT (a, b))
+  udf(count(DISTINCT 1)),
+  udf(count(DISTINCT null)),
+  udf(count(DISTINCT a)),
+  udf(count(DISTINCT b)),
+  udf(count(DISTINCT (a + b))),
+  udf(count(DISTINCT (a, b)))
 FROM testData
 -- !query 2 schema
-struct<count(DISTINCT 1):bigint,count(DISTINCT NULL):bigint,count(DISTINCT a):bigint,count(DISTINCT b):bigint,count(DISTINCT (a + b)):bigint,count(DISTINCT named_struct(a, a, b, b)):bigint>
+struct<udf(count(distinct 1)):string,udf(count(distinct null)):string,udf(count(distinct a)):string,udf(count(distinct b)):string,udf(count(distinct (a + b))):string,udf(count(distinct named_struct(a, a, b, b))):string>
 -- !query 2 output
 1	0	2	2	2	6

 -- !query 3
-SELECT count(a, b), count(b, a), count(testData.*) FROM testData
+SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*)) FROM testData
 -- !query 3 schema
-struct<count(a, b):bigint,count(b, a):bigint,count(a, b):bigint>
+struct<udf(count(a, b)):string,udf(count(b, a)):string,udf(count(a, b)):string>
 -- !query 3 output
 4	4	4

 -- !query 4
 SELECT
-  count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*)
+  udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*))
 FROM testData
 -- !query 4 schema
-struct<count(DISTINCT a, b):bigint,count(DISTINCT b, a):bigint,count(DISTINCT a, b):bigint,count(DISTINCT a, b):bigint>
+struct<udf(count(distinct a, b)):string,udf(count(distinct b, a)):string,udf(count(distinct a, b)):string,udf(count(distinct a, b)):string>
 -- !query 4 output
 3	3	3	3

```

</p>
</details>

## How was this patch tested?

Tested as guided in SPARK-27921.

Closes #25089 from vinodkc/br_Fix_SPARK-28275.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 09:39:53 +09:00
manu.zhang 06ac7d5966 [SPARK-27922][SQL][PYTHON][TESTS] Convert and port 'natural-join.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `natural-join.sql` to test UDFs following the combination guide in  [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

<details><summary>Diff results comparing to `natural-join.sql`</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.
sql.out
index 43f2f9a..53ef177 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out
 -27,7 +27,7  struct<>

 -- !query 2
-SELECT * FROM nt1 natural join nt2 where k = "one"
+SELECT * FROM nt1 natural join nt2 where udf(k) = "one"
 -- !query 2 schema
 struct<k:string,v1:int,v2:int>
 -- !query 2 output
 -36,7 +36,7  one   1       5

 -- !query 3
-SELECT * FROM nt1 natural left join nt2 order by v1, v2
+SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2
 -- !query 3 schema
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.
sql.out
index 43f2f9a..53ef177 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out
 -27,7 +27,7  struct<>

 -- !query 2
-SELECT * FROM nt1 natural join nt2 where k = "one"
+SELECT * FROM nt1 natural join nt2 where udf(k) = "one"
 -- !query 2 schema
 struct<k:string,v1:int,v2:int>
 -- !query 2 output
 -36,7 +36,7  one   1       5

 -- !query 3
-SELECT * FROM nt1 natural left join nt2 order by v1, v2
+SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2
 -- !query 3 schema
 struct<k:string,v1:int,v2:int>
```

</p>
</details>

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25088 from manuzhang/SPARK-27922.

Authored-by: manu.zhang <manu.zhang@vipshop.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 09:37:25 +09:00
Liang-Chi Hsieh 7858e534d3 [SPARK-28323][SQL][PYTHON] PythonUDF should be able to use in join condition
## What changes were proposed in this pull request?

There is a bug in `ExtractPythonUDFs` that produces wrong result attributes. It causes a failure when using `PythonUDF`s among multiple child plans, e.g., join. An example is using `PythonUDF`s in join condition.

```python
>>> left = spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)])
>>> right = spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)])
>>> f = udf(lambda a: a, IntegerType())
>>> df = left.join(right, [f("a") == f("b"), left.a1 == right.b1])
>>> df.collect()
19/07/10 12:20:49 ERROR Executor: Exception in task 5.0 in stage 0.0 (TID 5)
java.lang.ArrayIndexOutOfBoundsException: 1
        at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.genericGet(rows.scala:201)
        at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getAs(rows.scala:35)
        at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.isNullAt(rows.scala:36)
        at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.isNullAt$(rows.scala:36)
        at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.isNullAt(rows.scala:195)
        at org.apache.spark.sql.catalyst.expressions.JoinedRow.isNullAt(JoinedRow.scala:70)
        ...
```

## How was this patch tested?

Added test.

Closes #25091 from viirya/SPARK-28323.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-07-10 16:29:58 -07:00
Dongjoon Hyun a6506f0c8a [SPARK-28290][CORE][SQL] Use SslContextFactory.Server instead of SslContextFactory
## What changes were proposed in this pull request?

`SslContextFactory` is deprecated at Jetty 9.4 and we are using `9.4.18.v20190429`. This PR aims to replace it with `SslContextFactory.Server`.
- https://www.eclipse.org/jetty/javadoc/9.4.19.v20190610/org/eclipse/jetty/util/ssl/SslContextFactory.html
- https://www.eclipse.org/jetty/javadoc/9.3.24.v20180605/org/eclipse/jetty/util/ssl/SslContextFactory.html

```
[WARNING] /Users/dhyun/APACHE/spark/core/src/main/scala/org/apache/spark/SSLOptions.scala:71:
constructor SslContextFactory in class SslContextFactory is deprecated:
see corresponding Javadoc for more information.
[WARNING]       val sslContextFactory = new SslContextFactory()
[WARNING]                               ^
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #25067 from dongjoon-hyun/SPARK-28290.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 09:05:56 -07:00
Zhu, Lipeng b89c3de1a4 [SPARK-28310][SQL] Support (FIRST_VALUE|LAST_VALUE)(expr[ (IGNORE|RESPECT) NULLS]?) syntax
## What changes were proposed in this pull request?
According to the ANSI SQL 2011
![image](https://user-images.githubusercontent.com/698621/60855327-d01c6900-a235-11e9-9a1b-d438615a4673.png)

Below are Teradata, Oracle, Redshift which already support this grammar.

- Teradata - https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/SUwCpTupqmlBJvi2mipOaA
- Oracle - https://docs.oracle.com/en/database/oracle/oracle-database/18/sqlrf/FIRST_VALUE.html#GUID-D454EC3F-370C-4C64-9B11-33FCB10D95EC
- Redshift – https://docs.aws.amazon.com/redshift/latest/dg/r_WF_first_value.html

- Postgresql didn't implement this grammar:
https://www.postgresql.org/docs/devel/functions-window.html

  >The SQL standard defines a RESPECT NULLS or IGNORE NULLS option for lead, lag, first_value, last_value, and nth_value. This is not implemented in PostgreSQL: the behavior is always the same as the standard's default, namely RESPECT NULLS.

## How was this patch tested?
UT.

Closes #25082 from lipzhu/SPARK-28310.

Authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 07:41:05 -07:00
Yuming Wang 1b232671a8 [SPARK-28136][SQL][TEST] Port int8.sql
## What changes were proposed in this pull request?

This PR is to port int8.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int8.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/int8.out

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28137](https://issues.apache.org/jira/browse/SPARK-28137): Missing Data Type Formatting Functions
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027): Missing some mathematical operators

Also, found three inconsistent behavior:
[SPARK-26218](https://issues.apache.org/jira/browse/SPARK-28024): Throw exception on overflow for integers
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Spark SQL insert bad inputs to NULL
[SPARK-28028](https://issues.apache.org/jira/browse/SPARK-28028): Cast numeric to integral type need round
[SPARK-2659](https://issues.apache.org/jira/browse/SPARK-2659): HiveQL: Division operator should always perform fractional division, for example:
```sql
select 1/2;
```

## How was this patch tested?

N/A

Closes #24933 from wangyum/SPARK-28136.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-09 20:33:35 -07:00
Yuming Wang 019efaa375 [SPARK-28029][SQL][TEST] Port int2.sql
## What changes were proposed in this pull request?

This PR is to port int2.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int2.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/int2.out

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28023](https://issues.apache.org/jira/browse/SPARK-28023): Trim the string when cast string type to other types
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027): Add bitwise shift left/right operators

Also, found a bug:
[SPARK-28024](https://issues.apache.org/jira/browse/SPARK-28024): Incorrect value when out of range

Also, found three inconsistent behavior:
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Invalid input syntax for smallint throws exception at PostgreSQL
[SPARK-28028](https://issues.apache.org/jira/browse/SPARK-28028): Cast numeric to integral type need round
[SPARK-2659](https://issues.apache.org/jira/browse/SPARK-2659): HiveQL: Division operator should always perform fractional division, for example:
```sql
select 1/2;
```

## How was this patch tested?

N/A

Closes #24853 from wangyum/SPARK-28029.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-09 08:49:31 -07:00
Wenchen Fan 75ea02bb81 [SPARK-28250][SQL] QueryPlan#references should exclude producedAttributes
## What changes were proposed in this pull request?

This is a followup of the discussion in https://github.com/apache/spark/pull/24675#discussion_r286786053

`QueryPlan#references` is an important property. The `ColumnPrunning` rule relies on it.

Some query plan nodes have `Seq[Attribute]` parameter, which is used as its output attributes. For example, leaf nodes, `Generate`, `MapPartitionsInPandas`, etc. These nodes override `producedAttributes` to make `missingInputs` correct.

However, these nodes also need to override `references` to make column pruning work. This PR proposes to exclude `producedAttributes` from the default implementation of `QueryPlan#references`, so that we don't need to override `references` in all these nodes.

Note that, technically we can remove `producedAttributes` and always ask query plan nodes to override `references`. But I do find the code can be simpler with `producedAttributes` in some places, where there is a base class for some specific query plan nodes.

## How was this patch tested?

existing tests

Closes #25052 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-09 12:04:48 +09:00
HyukjinKwon fe3e34dda6 [SPARK-28273][SQL][PYTHON] Convert and port 'pgSQL/case.sql' into UDF test base
## What changes were proposed in this pull request?

This PR adds some tests converted from `pgSQL/case.sql'` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

This PR also contains two minor fixes:

1. Change name of Scala UDF from `UDF:name(...)` to `name(...)` to be consistent with Python'

2. Fix Scala UDF at `IntegratedUDFTestUtils.scala ` to handle `null` in strings.

<details><summary>Diff comparing to 'pgSQL/case.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out
index fa078d16d6d..55bef64338f 100644
--- a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out
 -115,7 +115,7  struct<>
 -- !query 13
 SELECT '3' AS `One`,
   CASE
-    WHEN 1 < 2 THEN 3
+    WHEN CAST(udf(1 < 2) AS boolean) THEN 3
   END AS `Simple WHEN`
 -- !query 13 schema
 struct<One:string,Simple WHEN:int>
 -126,10 +126,10  struct<One:string,Simple WHEN:int>
 -- !query 14
 SELECT '<NULL>' AS `One`,
   CASE
-    WHEN 1 > 2 THEN 3
+    WHEN 1 > 2 THEN udf(3)
   END AS `Simple default`
 -- !query 14 schema
-struct<One:string,Simple default:int>
+struct<One:string,Simple default:string>
 -- !query 14 output
 <NULL> NULL

 -137,17 +137,17  struct<One:string,Simple default:int>
 -- !query 15
 SELECT '3' AS `One`,
   CASE
-    WHEN 1 < 2 THEN 3
-    ELSE 4
+    WHEN udf(1) < 2 THEN udf(3)
+    ELSE udf(4)
   END AS `Simple ELSE`
 -- !query 15 schema
-struct<One:string,Simple ELSE:int>
+struct<One:string,Simple ELSE:string>
 -- !query 15 output
 3      3

 -- !query 16
-SELECT '4' AS `One`,
+SELECT udf('4') AS `One`,
   CASE
     WHEN 1 > 2 THEN 3
     ELSE 4
 -159,10 +159,10  struct<One:string,ELSE default:int>

 -- !query 17
-SELECT '6' AS `One`,
+SELECT udf('6') AS `One`,
   CASE
-    WHEN 1 > 2 THEN 3
-    WHEN 4 < 5 THEN 6
+    WHEN CAST(udf(1 > 2) AS boolean) THEN 3
+    WHEN udf(4) < 5 THEN 6
     ELSE 7
   END AS `Two WHEN with default`
 -- !query 17 schema
 -173,7 +173,7  struct<One:string,Two WHEN with default:int>

 -- !query 18
 SELECT '7' AS `None`,
-  CASE WHEN rand() < 0 THEN 1
+  CASE WHEN rand() < udf(0) THEN 1
   END AS `NULL on no matches`
 -- !query 18 schema
 struct<None:string,NULL on no matches:int>
 -182,36 +182,36  struct<None:string,NULL on no matches:int>

 -- !query 19
-SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END
+SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END
 -- !query 19 schema
-struct<CASE WHEN (1 = 0) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double>
+struct<CASE WHEN CAST(udf((1 = 0)) AS BOOLEAN) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double>
 -- !query 19 output
 1.0

 -- !query 20
-SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END
+SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END
 -- !query 20 schema
-struct<CASE WHEN (1 = 0) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double>
+struct<CASE WHEN (1 = 0) THEN (CAST(1 AS DOUBLE) / CAST(CAST(udf(0) AS DOUBLE) AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double>
 -- !query 20 output
 1.0

 -- !query 21
-SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl
+SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl
 -- !query 21 schema
-struct<CASE WHEN (i > 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double>
+struct<CASE WHEN (i > 100) THEN udf((cast(1 as double) / cast(0 as double))) ELSE udf(0) END:string>
 -- !query 21 output
-0.0
-0.0
-0.0
-0.0
+0
+0
+0
+0

 -- !query 22
-SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END
+SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END
 -- !query 22 schema
-struct<CASE WHEN (a = a) THEN 1 ELSE 2 END:int>
+struct<CASE WHEN (a = a) THEN udf(1) ELSE udf(2) END:string>
 -- !query 22 output
 1

 -283,7 +283,7  big

 -- !query 27
-SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4
+SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4
 -- !query 27 schema
 struct<i:int,f:double>
 -- !query 27 output
 -291,7 +291,7  struct<i:int,f:double>

 -- !query 28
-SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2
+SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2
 -- !query 28 schema
 struct<i:int,f:double>
 -- !query 28 output
 -299,10 +299,10  struct<i:int,f:double>

 -- !query 29
-SELECT COALESCE(a.f, b.i, b.j)
+SELECT udf(COALESCE(a.f, b.i, b.j))
   FROM CASE_TBL a, CASE2_TBL b
 -- !query 29 schema
-struct<coalesce(f, CAST(i AS DOUBLE), CAST(j AS DOUBLE)):double>
+struct<udf(coalesce(f, cast(i as double), cast(j as double))):string>
 -- !query 29 output
 -30.3
 -30.3
 -332,8 +332,8  struct<coalesce(f, CAST(i AS DOUBLE), CAST(j AS DOUBLE)):double>

 -- !query 30
 SELECT *
-  FROM CASE_TBL a, CASE2_TBL b
-  WHERE COALESCE(a.f, b.i, b.j) = 2
+   FROM CASE_TBL a, CASE2_TBL b
+   WHERE udf(COALESCE(a.f, b.i, b.j)) = 2
 -- !query 30 schema
 struct<i:int,f:double,i:int,j:int>
 -- !query 30 output
 -342,7 +342,7  struct<i:int,f:double,i:int,j:int>

 -- !query 31
-SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`,
+SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`,
   NULLIF(b.i, 4) AS `NULLIF(b.i,4)`
   FROM CASE_TBL a, CASE2_TBL b
 -- !query 31 schema
 -377,7 +377,7  struct<Five:string,NULLIF(a.i,b.i):int,NULLIF(b.i,4):int>
 -- !query 32
 SELECT '' AS `Two`, *
   FROM CASE_TBL a, CASE2_TBL b
-  WHERE COALESCE(f,b.i) = 2
+  WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean)
 -- !query 32 schema
 struct<Two:string,i:int,f:double,i:int,j:int>
 -- !query 32 output
 -388,15 +388,15  struct<Two:string,i:int,f:double,i:int,j:int>
 -- !query 33
 SELECT CASE
   (CASE vol('bar')
-    WHEN 'foo' THEN 'it was foo!'
-    WHEN vol(null) THEN 'null input'
+    WHEN udf('foo') THEN 'it was foo!'
+    WHEN udf(vol(null)) THEN 'null input'
     WHEN 'bar' THEN 'it was bar!' END
   )
-  WHEN 'it was foo!' THEN 'foo recognized'
-  WHEN 'it was bar!' THEN 'bar recognized'
-  ELSE 'unrecognized' END
+  WHEN udf('it was foo!') THEN 'foo recognized'
+  WHEN 'it was bar!' THEN udf('bar recognized')
+  ELSE 'unrecognized' END AS col
 -- !query 33 schema
-struct<CASE WHEN (CASE WHEN (UDF:vol(bar) = foo) THEN it was foo! WHEN (UDF:vol(bar) = UDF:vol(null)) THEN null input WHEN (UDF:vol(bar) = bar) THEN it was bar! END = it was foo!) THEN foo recognized WHEN (CASE WHEN (UDF:vol(bar) = foo) THEN it was foo! WHEN (UDF:vol(bar) = UDF:vol(null)) THEN null input WHEN (UDF:vol(bar) = bar) THEN it was bar! END = it was bar!) THEN bar recognized ELSE unrecognized END:string>
+struct<col:string>
 -- !query 33 output
 bar recognized
```

</p>
</details>

https://github.com/apache/spark/pull/25069 contains the same minor fixes as it's required to write the tests.

## How was this patch tested?

Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).

Closes #25070 from HyukjinKwon/SPARK-28273.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-09 10:50:07 +08:00
Dongjoon Hyun a5ff9221fc [SPARK-28308][CORE] CalendarInterval sub-second part should be padded before parsing
## What changes were proposed in this pull request?

The sub-second part of the interval should be padded before parsing. Currently, Spark gives a correct value only when there is 9 digits below `.`.
```
spark-sql> select interval '0 0:0:0.123456789' day to second;
interval 123 milliseconds 456 microseconds

spark-sql> select interval '0 0:0:0.12345678' day to second;
interval 12 milliseconds 345 microseconds

spark-sql> select interval '0 0:0:0.1234' day to second;
interval 1 microseconds
```

## How was this patch tested?

Pass the Jenkins with the fixed test cases.

Closes #25079 from dongjoon-hyun/SPARK-28308.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-08 19:40:41 -07:00
Tony Zhang 20469d43eb [SPARK-28189][SQL] Use semanticEquals in Dataset drop method for attributes comparison
## What changes were proposed in this pull request?

In Dataset drop(col: Column) method, the `equals` comparison method was used instead of `semanticEquals`, which caused the problem of abnormal case-sensitivity behavior. When attributes of LogicalPlan are checked for equality, `semanticEquals` should be used instead.

A similar PR I referred to: https://github.com/apache/spark/pull/22713 created by mgaido91

## How was this patch tested?

- Added new unit test case in DataFrameSuite
- ./build/sbt "testOnly org.apache.spark.sql.*"
- The python code from ticket reporter at https://issues.apache.org/jira/browse/SPARK-28189

Closes #25055 from Tonix517/SPARK-28189.

Authored-by: Tony Zhang <tony.zhang@uber.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-06 21:39:04 -07:00
Yuming Wang 51379b731d [SPARK-28020][SQL][TEST] Port date.sql
## What changes were proposed in this pull request?

This PR is to port date.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/date.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/date.out

When porting the test cases, found four PostgreSQL specific features that do not exist in Spark SQL:

[SPARK-28017](https://issues.apache.org/jira/browse/SPARK-28017): Enhance EXTRACT/DATE_TRUNC
[SPARK-28141](https://issues.apache.org/jira/browse/SPARK-28141): Date type can not accept special values
[SPARK-28253](https://issues.apache.org/jira/browse/SPARK-28253):  Date type have different low value and high value
[SPARK-28259](https://issues.apache.org/jira/browse/SPARK-28259): Date/Time Output Styles and Date Order Conventions

Also, found a bug:
[SPARK-28015](https://issues.apache.org/jira/browse/SPARK-28015): Invalid date formats should throw an exception

Also, found a inconsistent behavior:
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Invalid date throw an exception bug Spark SQL returns `NULL`, for example: 30bcebbdcf/src/test/regress/expected/date.out (L13-L14)

## How was this patch tested?

N/A

Closes #24850 from wangyum/SPARK-28020.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-05 18:30:25 -07:00
Yuming Wang 4caf81a48f [SPARK-28093][SQL][FOLLOW-UP] Update trim function behavior changes to migration guide
## What changes were proposed in this pull request?

We changed our non-standard syntax for `trim` function  in #24902 from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. This pr update the migration guide.

I checked various databases(PostgreSQL, Teradata, Vertica, Oracle, DB2, SQL Server 2019, MySQL, Hive, Presto) and it seems that only PostgreSQL and Presto support this non-standard syntax.
**PostgreSQL**:
```sql
postgres=#  select substr(version(), 0, 16), trim('yxTomxx', 'x');
     substr      | btrim
-----------------+-------
 PostgreSQL 11.3 | yxTom
(1 row)
```
**Presto**:
```sql
presto> select trim('yxTomxx', 'x');
 _col0
-------
 yxTom
(1 row)
```

## How was this patch tested?

manual tests

Closes #24948 from wangyum/SPARK-28093-FOLLOW-UP-DOCS.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-05 17:55:54 -07:00
Peter Toth 1272df29fe [SPARK-28002][SQL][FOLLOWUP] Fix duplicate CTE error message and add more test cases
## What changes were proposed in this pull request?

This PR adds some more WITH test cases as a follow-up to https://github.com/apache/spark/pull/24842

## How was this patch tested?

Add new UTs.

Closes #24949 from peter-toth/SPARK-28002-follow-up.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-05 11:42:01 -07:00
Yuming Wang d493a1f6bf [SPARK-27898][SQL] Support 4 date operators(date + integer, integer + date, date - integer and date - date)
## What changes were proposed in this pull request?

This pr add support 4 PostgreSQL's date operators(date + integer, integer + date, date - integer and date - date):

Operator | Example | Result
-- | -- | --
\+ | date '2001-09-28' + 7 | date '2001-10-05'
\+ | 7 + date '2001-09-28' | date '2001-10-05'
\- | date '2001-10-01' - 7 | date '2001-09-24'
\- | date '2001-10-01' - date '2001-09-28' | integer '3' (days)

Most databases support `date - date` operation, where PostgreSQL, Vertica, Teradata, Oracle and DB2 returns `Integer` type, Hive and Presto returns `Interval` type, MySQL returns unexpected value, and SQL Server does not support `date - date` operation.

**PostgreSQL**:
```sql
postgres=# select substr(version(), 0, 16), date '2001-09-28' + 7, 7 + date '2001-09-28', date '2001-10-01' - 7, date '2001-10-01' - date '2001-09-28';
     substr      |  ?column?  |  ?column?  |  ?column?  | ?column?
-----------------+------------+------------+------------+----------
 PostgreSQL 11.3 | 2001-10-05 | 2001-10-05 | 2001-09-24 |        3
(1 row)
```
**Vertica**:
```sql
dbadmin=> select version(), date '2001-09-28' + 7, 7 + date '2001-09-28', date '2001-10-01' - 7, date '2001-10-01' - date '2001-09-28';
              version               |  ?column?  |  ?column?  |  ?column?  | ?column?
------------------------------------+------------+------------+------------+----------
 Vertica Analytic Database v9.1.1-0 | 2001-10-05 | 2001-10-05 | 2001-09-24 |        3
(1 row)
```
**Teradata**:
![image](https://user-images.githubusercontent.com/5399861/59563983-8ba50f80-9073-11e9-821a-9f85b5f2820c.png)

**Oracle**:
![image](https://user-images.githubusercontent.com/5399861/59563928-e68a3700-9072-11e9-8663-e28231a7ac83.png)
**DB2**:
![image](https://user-images.githubusercontent.com/5399861/59564326-fbb59480-9077-11e9-9520-e12ec3e59b0c.png)
**Hive**:
```sql
hive> select version(),  date '2001-10-01' - date '2001-09-28';
OK
3.1.1 rf4e0529634b6231a0072295da48af466cf2f10b7	3 00:00:00.000000000
Time taken: 2.038 seconds, Fetched: 1 row(s)
```
**Presto**:
```sql
presto> select  date '2001-10-01' - date '2001-09-28';
     _col0
----------------
 3 00:00:00.000
(1 row)
```
**MySQL**:
```SQL
mysql> SELECT version(), date '2001-10-01' - date '2001-09-28';
+-----------+---------------------------------------+
| version() | date '2001-10-01' - date '2001-09-28' |
+-----------+---------------------------------------+
| 5.7.26    |                                    73 |
+-----------+---------------------------------------+
1 row in set (0.00 sec)
```

More details:
https://www.postgresql.org/docs/12/functions-datetime.html

## How was this patch tested?

unit tests

Closes #24755 from wangyum/Add4DateOperators.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-05 10:01:43 -07:00
Mick Jermsurawong 683e270c16 [SPARK-28200][SQL] Decimal overflow handling in ExpressionEncoder
## What changes were proposed in this pull request?

- Currently, `ExpressionEncoder` does not handle bigdecimal overflow. Round-tripping overflowing java/scala BigDecimal/BigInteger returns null.
  - The serializer encode java/scala BigDecimal to to sql Decimal, which still has the underlying data to the former.
  - When writing out to UnsafeRow, `changePrecision` will be false and row has null value.
24e1e41648/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java (L202-L206)
- In [SPARK-23179](https://github.com/apache/spark/pull/20350), an option to throw exception on decimal overflow was introduced.
- This PR adds the option in `ExpressionEncoder` to throw when detecting overflowing BigDecimal/BigInteger before its corresponding Decimal gets written to Row. This gives a consistent behavior between decimal arithmetic on sql expression (DecimalPrecision), and getting decimal from dataframe (RowEncoder)

Thanks to mgaido91 for the very first PR `SPARK-23179` and follow-up discussion on this change.
Thanks to JoshRosen for working with me on this.

## How was this patch tested?

added unit tests

Closes #25016 from mickjermsurawong-stripe/SPARK-28200.

Authored-by: Mick Jermsurawong <mickjermsurawong@stripe.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-05 22:05:26 +08:00
Yuming Wang e299f62ec5 [SPARK-28241][SQL] Show metadata operations on ThriftServerTab
## What changes were proposed in this pull request?

This pr add support show metadata operations on ThriftServerTab.

## How was this patch tested?

manual tests:
![image](https://user-images.githubusercontent.com/5399861/60579741-4cd2c180-9db6-11e9-822a-0433be509b67.png)

Closes #25043 from wangyum/SPARK-28241.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: herman <herman@databricks.com>
2019-07-05 12:24:50 +02:00
HyukjinKwon 5c55812400 [SPARK-28198][PYTHON][FOLLOW-UP] Rename mapPartitionsInPandas to mapInPandas with a separate evaluation type
## What changes were proposed in this pull request?

This PR proposes to rename `mapPartitionsInPandas` to `mapInPandas` with a separate evaluation type .

Had an offline discussion with rxin, mengxr and cloud-fan

The reason is basically:

1. `SCALAR_ITER` doesn't make sense with `mapPartitionsInPandas`.
2. It cannot share the same Pandas UDF, for instance, at `select` and `mapPartitionsInPandas` unlike `GROUPED_AGG` because iterator's return type is different.
3. `mapPartitionsInPandas` -> `mapInPandas` - see https://github.com/apache/spark/pull/25044#issuecomment-508298552 and https://github.com/apache/spark/pull/25044#issuecomment-508299764

Renaming `SCALAR_ITER` as `MAP_ITER` is abandoned due to 2. reason.

For `XXX_ITER`, it might have to have a different interface in the future if we happen to add other versions of them. But this is an orthogonal topic with `mapPartitionsInPandas`.

## How was this patch tested?

Existing tests should cover.

Closes #25044 from HyukjinKwon/SPARK-28198.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-05 09:22:41 +09:00
Peter Toth 4ed88b32ad [SPARK-28251][SQL] Fix error message of inserting into a non-existing table
## What changes were proposed in this pull request?

Before this PR inserting into a non-existing table returned a weird error message:
```
sql("INSERT INTO test VALUES (1)").show
org.apache.spark.sql.AnalysisException: unresolved operator 'InsertIntoTable 'UnresolvedRelation [test], false, false;;
'InsertIntoTable 'UnresolvedRelation [test], false, false
+- LocalRelation [col1#4]
```
after this PR the error message becomes:
```
org.apache.spark.sql.AnalysisException: Table not found: test;;
'InsertIntoTable 'UnresolvedRelation [test], false, false
+- LocalRelation [col1#0]
```

## How was this patch tested?

Added a new UT.

Closes #25054 from peter-toth/SPARK-28251.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-04 12:32:18 -07:00
Peter Toth cad440d1f5 [SPARK-19799][SQL] Support WITH clause in subqueries
## What changes were proposed in this pull request?

This PR  adds support of `WITH` clause within a subquery so this query becomes valid:
  ```
  SELECT max(c) FROM (
    WITH t AS (SELECT 1 AS c)
    SELECT * FROM t
  )
 ```

## How was this patch tested?

Added new UTs.

Closes #24831 from peter-toth/SPARK-19799-2.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-04 07:34:02 -07:00
Carson Wang cec6a32904 [SPARK-28177][SQL] Adjust post shuffle partition number in adaptive execution
## What changes were proposed in this pull request?
This is to implement a ReduceNumShufflePartitions rule in the new adaptive execution framework introduced in #24706. This rule is used to adjust the post shuffle partitions based on the map output statistics.

## How was this patch tested?
Added ReduceNumShufflePartitionsSuite

Closes #24978 from carsonwang/reduceNumShufflePartitions.

Authored-by: Carson Wang <carson.wang@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-04 16:03:04 +08:00
Yuming Wang fb718d26cf [SPARK-28216][SQL][TEST] Add getLocalDirSize to SQLTestUtils
## What changes were proposed in this pull request?

This pr add calculate local directory size to `SQLTestUtils`.

We can avoid these changes after this pr:
![image](https://user-images.githubusercontent.com/5399861/60386910-66ca8680-9ace-11e9-8d52-e1eea38e324a.png)

## How was this patch tested?

Existing test

Closes #25014 from wangyum/SPARK-28216.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-03 12:48:19 -07:00
Yesheng Ma 74f1176311 [SPARK-27815][SQL] Predicate pushdown in one pass for cascading joins
## What changes were proposed in this pull request?

This PR makes the predicate pushdown logic in catalyst optimizer more efficient by unifying two existing rules `PushdownPredicates` and `PushPredicateThroughJoin`. Previously pushing down a predicate for queries such as `Filter(Join(Join(Join)))` requires n steps. This patch essentially reduces this to a single pass.

To make this actually work, we need to unify a few rules such as `CombineFilters`, `PushDownPredicate` and `PushDownPrdicateThroughJoin`. Otherwise cases such as `Filter(Join(Filter(Join)))` still requires several passes to fully push down predicates. This unification is done by composing several partial functions, which makes a minimal code change and can reuse existing UTs.

Results show that this optimization can improve the catalyst optimization time by 16.5%. For queries with more joins, the performance is even better. E.g., for TPC-DS q64, the performance boost is 49.2%.

## How was this patch tested?
Existing UTs + new a UT for the new rule.

Closes #24956 from yeshengm/fixed-point-opt.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-03 09:01:16 -07:00
Yuming Wang 70b1a10a26 [SPARK-28077][SQL][FOLLOW-UP] Add PLACING to ansiNonReserved
## What changes were proposed in this pull request?

This pr add `PLACING` to `ansiNonReserved` and add `overlay` and `placing` to `TableIdentifierParserSuite`.

## How was this patch tested?

N/A

Closes #25013 from wangyum/SPARK-28077.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-03 08:47:30 -07:00
Liang-Chi Hsieh 913ab4b9fd [SPARK-28156][SQL] Self-join should not miss cached view
## What changes were proposed in this pull request?

The issue is when self-join a cached view, only one side of join uses cached relation. The cause is in `ResolveReferences` we do deduplicate for a view to have new output attributes. Then in `AliasViewChild`, the rule adds extra project under a view. So it breaks cache matching.

The fix is when dedup, we only dedup a view which has output different to its child plan. Otherwise, we dedup on the view's child plan.

```scala
val df = Seq.tabulate(5) { x => (x, x + 1, x + 2, x + 3) }.toDF("a", "b", "c", "d")
df.write.mode("overwrite").format("orc").saveAsTable("table1")

sql("drop view if exists table1_vw")
sql("create view table1_vw as select * from table1")

val cachedView = sql("select a, b, c, d from table1_vw")

cachedView.createOrReplaceTempView("cachedview")
cachedView.persist()

val queryDf = sql(
  s"""select leftside.a, leftside.b
      |from cachedview leftside
      |join cachedview rightside
      |on leftside.a = rightside.a
    """.stripMargin)
```

Query plan before this PR:
```scala
== Physical Plan ==
*(2) Project [a#12664, b#12665]
+- *(2) BroadcastHashJoin [a#12664], [a#12660], Inner, BuildRight
   :- *(2) Filter isnotnull(a#12664)
   :  +- *(2) InMemoryTableScan [a#12664, b#12665], [isnotnull(a#12664)]
   :        +- InMemoryRelation [a#12664, b#12665, c#12666, d#12667], StorageLevel(disk, memory, deserialized, 1 replicas)
   :              +- *(1) FileScan orc default.table1[a#12660,b#12661,c#12662,d#12663] Batched: true, DataFilters: [], Format: ORC, Location: InMemoryF
ileIndex[file:/Users/viirya/repos/spark-1/sql/core/spark-warehouse/org.apache.spark.sql...., PartitionFilters: [], PushedFilters: [], ReadSchema: struc
t<a:int,b:int,c:int,d:int>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)))
      +- *(1) Project [a#12660]
         +- *(1) Filter isnotnull(a#12660)
            +- *(1) FileScan orc default.table1[a#12660] Batched: true, DataFilters: [isnotnull(a#12660)], Format: ORC, Location: InMemoryFileIndex[fil
e:/Users/viirya/repos/spark-1/sql/core/spark-warehouse/org.apache.spark.sql...., PartitionFilters: [], PushedFilters: [IsNotNull(a)], ReadSchema: struc
t<a:int>
```

Query plan after this PR:
```scala
== Physical Plan ==
*(2) Project [a#12664, b#12665]
+- *(2) BroadcastHashJoin [a#12664], [a#12692], Inner, BuildRight
   :- *(2) Filter isnotnull(a#12664)
   :  +- *(2) InMemoryTableScan [a#12664, b#12665], [isnotnull(a#12664)]
   :        +- InMemoryRelation [a#12664, b#12665, c#12666, d#12667], StorageLevel(disk, memory, deserialized, 1 replicas)
   :              +- *(1) FileScan orc default.table1[a#12660,b#12661,c#12662,d#12663] Batched: true, DataFilters: [], Format: ORC, Location: InMemoryFileIndex[file:/Users/viirya/repos/spark-1/sql/core/spark-warehouse/org.apache.spark.sql...., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:int,b:int,c:int,d:int>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
      +- *(1) Filter isnotnull(a#12692)
         +- *(1) InMemoryTableScan [a#12692], [isnotnull(a#12692)]
               +- InMemoryRelation [a#12692, b#12693, c#12694, d#12695], StorageLevel(disk, memory, deserialized, 1 replicas)
                     +- *(1) FileScan orc default.table1[a#12660,b#12661,c#12662,d#12663] Batched: true, DataFilters: [], Format: ORC, Location: InMemoryFileIndex[file:/Users/viirya/repos/spark-1/sql/core/spark-warehouse/org.apache.spark.sql...., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:int,b:int,c:int,d:int>
```

## How was this patch tested?

Added test.

Closes #24960 from viirya/SPARK-28156.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-03 21:21:31 +08:00
Yuming Wang ea0303063f [SPARK-28167][SQL] Show global temporary view in database tool
## What changes were proposed in this pull request?

This pr add support show global temporary view and local temporary view in database tool.

TODO: Database tools should support show temporary views because it's schema is null.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/60392266-a5455d00-9b31-11e9-92c8-88a8e6c2aec3.png)

Closes #24972 from wangyum/SPARK-28167.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-03 00:01:05 -07:00
Jose Torres 4ebff5b6d6 [SPARK-28223][SS] stream-stream joins should fail unsupported checker in update mode
## What changes were proposed in this pull request?

Right now they fail only for inner joins, because we implemented the check when that was the only supported type.

## How was this patch tested?

new unit test

Closes #25023 from jose-torres/changevalidation.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Jose Torres <torres.joseph.f+github@gmail.com>
2019-07-02 09:59:11 -07:00
Dooyoung Hwang 2ff1ac5d9f [SPARK-25353][SQL] executeTake in SparkPlan is modified to avoid unnecessary decoding.
## What changes were proposed in this pull request?
In some cases, executeTake in SparkPlan could decode more than necessary.

For example, in case of below odd/even number partitioning, total row's count from partitions will be 100, although it is limited with 51. And 'executeTake' in SparkPlan decodes all of them, "49" rows of which are unnecessarily decoded.

```scala
spark.sparkContext.parallelize((0 until 100).map(i => (i, 1))).toDF()
      .repartitionByRange(2, $"_1" % 2).limit(51).collect()
```

By using a iterator of the scalar collection, we can make ensure that at most n rows are decoded.

## How was this patch tested?
Existing unit tests that call limit function of DataFrame.

testOnly *SQLQuerySuite
testOnly *DataFrameSuite

Closes #22347 from Dooyoung-Hwang/refactor_execute_take.

Authored-by: Dooyoung Hwang <dooyoung.hwang@sk.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-02 20:55:24 +08:00
Liang-Chi Hsieh f1486742fa [SPARK-28054][SQL][FOLLOW-UP] Fix error when insert Hive partitioned table dynamically where partition name is upper case
## What changes were proposed in this pull request?

This is a small follow-up for SPARK-28054 to fix wrong indent and use `withSQLConf` as suggested by gatorsmile.

## How was this patch tested?

Existing tests.

Closes #24971 from viirya/SPARK-28054-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-02 14:57:24 +09:00
HyukjinKwon 02f4763286 [SPARK-28198][PYTHON] Add mapPartitionsInPandas to allow an iterator of DataFrames
## What changes were proposed in this pull request?

This PR proposes to add `mapPartitionsInPandas` API to DataFrame by using existing `SCALAR_ITER` as below:

1. Filtering via setting the column

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType

df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))

pandas_udf(df.schema, PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
    for pdf in iterator:
        yield pdf[pdf.id == 1]

df.mapPartitionsInPandas(filter_func).show()
```

```
+---+---+
| id|age|
+---+---+
|  1| 21|
+---+---+
```

2. `DataFrame.loc`

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
import pandas as pd

df = spark.createDataFrame([['aa'], ['bb'], ['cc'], ['aa'], ['aa'], ['aa']], ["value"])

pandas_udf(df.schema, PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
    for pdf in iterator:
        yield pdf.loc[pdf.value.str.contains('^a'), :]

df.mapPartitionsInPandas(filter_func).show()
```

```
+-----+
|value|
+-----+
|   aa|
|   aa|
|   aa|
|   aa|
+-----+
```

3. `pandas.melt`

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
import pandas as pd

df = spark.createDataFrame(
    pd.DataFrame({'A': {0: 'a', 1: 'b', 2: 'c'},
                  'B': {0: 1, 1: 3, 2: 5},
                  'C': {0: 2, 1: 4, 2: 6}}))

pandas_udf("A string, variable string, value long", PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
    for pdf in iterator:
        import pandas as pd
        yield pd.melt(pdf, id_vars=['A'], value_vars=['B', 'C'])

df.mapPartitionsInPandas(filter_func).show()
```

```
+---+--------+-----+
|  A|variable|value|
+---+--------+-----+
|  a|       B|    1|
|  a|       C|    2|
|  b|       B|    3|
|  b|       C|    4|
|  c|       B|    5|
|  c|       C|    6|
+---+--------+-----+
```

The current limitation of `SCALAR_ITER` is that it doesn't allow different length of result, which is pretty critical in practice - for instance, we cannot simply filter by using Pandas APIs but we merely just map N to N. This PR allows map N to M like flatMap.

This API mimics the way of `mapPartitions` but keeps API shape of `SCALAR_ITER` by allowing different results.

### How does this PR implement?

This PR adds mimics both `dapply` with Arrow optimization and Grouped Map Pandas UDF. At Python execution side, it reuses existing `SCALAR_ITER` code path.

Therefore, externally, we don't introduce any new type of Pandas UDF but internally we use another evaluation type code `205` (`SQL_MAP_PANDAS_ITER_UDF`).

This approach is similar with Pandas' Windows function implementation with Grouped Aggregation Pandas UDF functions - internally we have `203` (`SQL_WINDOW_AGG_PANDAS_UDF`) but externally we just share the same `GROUPED_AGG`.

## How was this patch tested?

Manually tested and unittests were added.

Closes #24997 from HyukjinKwon/scalar-udf-iter.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-02 10:54:16 +09:00
Gabor Somogyi 0a4f985ca0 [SPARK-23098][SQL] Migrate Kafka Batch source to v2.
## What changes were proposed in this pull request?

Kafka batch data source is using v1 at the moment. In the PR I've migrated to v2. Majority of the change is moving code.

What this PR contains:
* useV1Sources usage fixed in `DataFrameReader` and `DataFrameWriter`
* `KafkaBatch` added to handle DSv2 batch reading
* `KafkaBatchWrite` added to handle DSv2 batch writing
* `KafkaBatchPartitionReader` extracted to share between batch and microbatch
* `KafkaDataWriter` extracted to share between batch, microbatch and continuous
* Batch related source/sink tests are now executing on v1 and v2 connectors
* Couple of classes hidden now, functions moved + couple of minor fixes

## How was this patch tested?

Existing + added unit tests.

Closes #24738 from gaborgsomogyi/SPARK-23098.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-02 09:47:30 +08:00
Gengliang Wang 3ae531ebb9 [SPARK-28205][SQL] useV1SourceList configuration should be for all data sources
## What changes were proposed in this pull request?

In the migration PR of Kafka V2: ac16c9a9ef (r298470645)
We find that the useV1SourceList configuration(spark.sql.sources.read.useV1SourceList and spark.sql.sources.write.useV1SourceList) should be for all data sources, instead of file source V2 only.

This PR is to fix it in DataFrameWriter/DataFrameReader.
## How was this patch tested?

Unit test

Closes #25004 from gengliangwang/reviseUseV1List.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-01 14:02:42 +08:00
Marco Gaido bc4a676b27 [SPARK-28201][SQL] Revisit MakeDecimal behavior on overflow
## What changes were proposed in this pull request?

In SPARK-23179, it has been introduced a flag to control the behavior in case of overflow on decimals. The behavior is: returning `null` when `spark.sql.decimalOperations.nullOnOverflow` (default and traditional Spark behavior); throwing an `ArithmeticException` if that conf is false (according to SQL standards, other DBs behavior).

`MakeDecimal` so far had an ambiguous behavior. In case of codegen mode, it returned `null` as the other operators, but in interpreted mode, it was throwing an `IllegalArgumentException`.

The PR aligns `MakeDecimal`'s behavior with the one of other operators as defined in SPARK-23179. So now both modes return `null` or throw `ArithmeticException` according to `spark.sql.decimalOperations.nullOnOverflow`'s value.

Credits for this PR to mickjermsurawong-stripe who pointed out the wrong behavior in #20350.

## How was this patch tested?

improved UTs

Closes #25010 from mgaido91/SPARK-28201.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-01 11:54:58 +08:00
Yuming Wang 24e1e41648 [SPARK-28196][SQL] Add a new listTables and listLocalTempViews APIs for SessionCatalog
## What changes were proposed in this pull request?

This pr add two API for [SessionCatalog](df4cb471c9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala):
```scala
def listTables(db: String, pattern: String, includeLocalTempViews: Boolean): Seq[TableIdentifier]

def listLocalTempViews(pattern: String): Seq[TableIdentifier]
```
Because in some cases `listTables` does not need local temporary view and sometimes only need list local temporary view.

## How was this patch tested?

unit tests

Closes #24995 from wangyum/SPARK-28196.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-29 18:36:36 -07:00
Yuming Wang e0e2144ca6 [SPARK-28184][SQL][TEST] Avoid creating new sessions in SparkMetadataOperationSuite
## What changes were proposed in this pull request?

To make the #24972 change smaller. This pr improves `SparkMetadataOperationSuite` to avoid creating new sessions when getSchemas/getTables/getColumns.

## How was this patch tested?

N/A

Closes #24985 from wangyum/SPARK-28184.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-29 18:29:37 -07:00
wangguangxin.cn 73183b3c8c [SPARK-11412][SQL] Support merge schema for ORC
## What changes were proposed in this pull request?

Currently, ORC's `inferSchema` is implemented as randomly choosing one ORC file and reading its schema.

This PR follows the behavior of Parquet, it implements merge schemas logic by reading all ORC files in parallel through a spark job.

Users can enable merge schema by `spark.read.orc("xxx").option("mergeSchema", "true")` or by setting `spark.sql.orc.mergeSchema` to `true`, the prior one has higher priority.

## How was this patch tested?
tested by UT OrcUtilsSuite.scala

Closes #24043 from WangGuangxin/SPARK-11412.

Lead-authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Co-authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-29 17:08:31 -07:00
HyukjinKwon facf9c30a2 [SPARK-28204][SQL][TESTS] Make separate two test cases for column pruning in binary files
## What changes were proposed in this pull request?

SPARK-27534 missed to address my own comments at https://github.com/WeichenXu123/spark/pull/8
It's better to push this in since the codes are already cleaned up.

## How was this patch tested?

Unittests fixed

Closes #25003 from HyukjinKwon/SPARK-27534.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-29 14:05:23 +09:00
Robert (Bobby) Evans c341de8b3e [SPARK-27945][SQL] Minimal changes to support columnar processing
## What changes were proposed in this pull request?

This is the first part of [SPARK-27396](https://issues.apache.org/jira/browse/SPARK-27396).  This is the minimum set of changes necessary to support a pluggable back end for columnar processing.  Follow on JIRAs would cover removing some of the duplication between functionality in this patch and functionality currently covered by things like ColumnarBatchScan.

## How was this patch tested?

I added in a new unit test to cover new code not really covered in other places.

I also did manual testing by implementing two plugins/extensions that take advantage of the new APIs to allow for columnar processing for some simple queries.  One version runs on the [CPU](https://gist.github.com/revans2/c3cad77075c4fa5d9d271308ee2f1b1d).  The other version run on a GPU, but because it has unreleased dependencies I will not include a link to it yet.

The CPU version I would expect to add in as an example with other documentation in a follow on JIRA

This is contributed on behalf of NVIDIA Corporation.

Closes #24795 from revans2/columnar-basic.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-06-28 14:00:12 -05:00
gengjiaan 832ff87918 [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.
## What changes were proposed in this pull request?

The `OVERLAY` function is a `ANSI` `SQL`.
For example:
```
SELECT OVERLAY('abcdef' PLACING '45' FROM 4);

SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5);

SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0);

SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4);
```
The results of the above four `SQL` are:
```
abc45f
yabadaba
yabadabadoo
bubba
```

Note: If the input string is null, then the result is null too.

There are some mainstream database support the syntax.
**PostgreSQL:**
https://www.postgresql.org/docs/11/functions-string.html

**Vertica:** https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/String/OVERLAY.htm?zoom_highlight=overlay

**Oracle:**
https://docs.oracle.com/en/database/oracle/oracle-database/19/arpls/UTL_RAW.html#GUID-342E37E7-FE43-4CE1-A0E9-7DAABD000369

**DB2:**
https://www.ibm.com/support/knowledgecenter/SSGMCP_5.3.0/com.ibm.cics.rexx.doc/rexx/overlay.html

There are some show of the PR on my production environment.
```
spark-sql> SELECT OVERLAY('abcdef' PLACING '45' FROM 4);
abc45f
Time taken: 6.385 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5);
yabadaba
Time taken: 0.191 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0);
yabadabadoo
Time taken: 0.186 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4);
bubba
Time taken: 0.151 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING '45' FROM 4);
NULL
Time taken: 0.22 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING 'daba' FROM 5);
NULL
Time taken: 0.157 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING 'daba' FROM 5 FOR 0);
NULL
Time taken: 0.254 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING 'ubb' FROM 2 FOR 4);
NULL
Time taken: 0.159 seconds, Fetched 1 row(s)
```

## How was this patch tested?

Exists UT and new UT.

Closes #24918 from beliefer/ansi-sql-overlay.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2019-06-28 19:13:08 +09:00
Yuming Wang 410a898cf9 [SPARK-28179][SQL] Avoid hard-coded config: spark.sql.globalTempDatabase
## What changes were proposed in this pull request?

Avoid hard-coded config: `spark.sql.globalTempDatabase`.

## How was this patch tested?

N/A

Closes #24979 from wangyum/SPARK-28179.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-28 10:42:35 +09:00
Wenchen Fan cded421aeb [SPARK-27871][SQL] LambdaVariable should use per-query unique IDs instead of globally unique IDs
## What changes were proposed in this pull request?

For simplicity, all `LambdaVariable`s are globally unique, to avoid any potential conflicts. However, this causes a perf problem: we can never hit codegen cache for encoder expressions that deal with collections (which means they contain `LambdaVariable`).

To overcome this problem, `LambdaVariable` should have per-query unique IDs. This PR does 2 things:
1. refactor `LambdaVariable` to carry an ID, so that it's easier to change the ID.
2. add an optimizer rule to reassign `LambdaVariable` IDs, which are per-query unique.

## How was this patch tested?

new tests

Closes #24735 from cloud-fan/dataset.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-27 11:34:47 -07:00
Marco Gaido 3139d642fa [SPARK-23179][SQL] Support option to throw exception if overflow occurs during Decimal arithmetic
## What changes were proposed in this pull request?

SQL ANSI 2011 states that in case of overflow during arithmetic operations, an exception should be thrown. This is what most of the SQL DBs do (eg. SQLServer, DB2). Hive currently returns NULL (as Spark does) but HIVE-18291 is open to be SQL compliant.

The PR introduce an option to decide which behavior Spark should follow, ie. returning NULL on overflow or throwing an exception.

## How was this patch tested?

added UTs

Closes #20350 from mgaido91/SPARK-23179.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-27 19:02:07 +08:00
Bryan Cutler c277afb12b [SPARK-27992][PYTHON] Allow Python to join with connection thread to propagate errors
## What changes were proposed in this pull request?

Currently with `toLocalIterator()` and `toPandas()` with Arrow enabled, if the Spark job being run in the background serving thread errors, it will be caught and sent to Python through the PySpark serializer.
This is not the ideal solution because it is only catch a SparkException, it won't handle an error that occurs in the serializer, and each method has to have it's own special handling to propagate the error.

This PR instead returns the Python Server object along with the serving port and authentication info, so that it allows the Python caller to join with the serving thread. During the call to join, the serving thread Future is completed either successfully or with an exception. In the latter case, the exception will be propagated to Python through the Py4j call.

## How was this patch tested?

Existing tests

Closes #24834 from BryanCutler/pyspark-propagate-server-error-SPARK-27992.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-26 13:05:41 -07:00
Josh Rosen d83f84a122 [SPARK-27676][SQL][SS] InMemoryFileIndex should respect spark.sql.files.ignoreMissingFiles
## What changes were proposed in this pull request?

Spark's `InMemoryFileIndex` contains two places where `FileNotFound` exceptions are caught and logged as warnings (during [directory listing](bcd3b61c4b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala (L274)) and [block location lookup](bcd3b61c4b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala (L333))). This logic was added in #15153 and #21408.

I think that this is a dangerous default behavior because it can mask bugs caused by race conditions (e.g. overwriting a table while it's being read) or S3 consistency issues (there's more discussion on this in the [JIRA ticket](https://issues.apache.org/jira/browse/SPARK-27676)). Failing fast when we detect missing files is not sufficient to make concurrent table reads/writes or S3 listing safe (there are other classes of eventual consistency issues to worry about), but I think it's still beneficial to throw exceptions and fail-fast on the subset of inconsistencies / races that we _can_ detect because that increases the likelihood that an end user will notice the problem and investigate further.

There may be some cases where users _do_ want to ignore missing files, but I think that should be an opt-in behavior via the existing `spark.sql.files.ignoreMissingFiles` flag (the current behavior is itself race-prone because a file might be be deleted between catalog listing and query execution time, triggering FileNotFoundExceptions on executors (which are handled in a way that _does_ respect `ignoreMissingFIles`)).

This PR updates `InMemoryFileIndex` to guard the log-and-ignore-FileNotFoundException behind the existing `spark.sql.files.ignoreMissingFiles` flag.

**Note**: this is a change of default behavior, so I think it needs to be mentioned in release notes.

## How was this patch tested?

New unit tests to simulate file-deletion race conditions, tested with both values of the `ignoreMissingFIles` flag.

Closes #24668 from JoshRosen/SPARK-27676.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-26 09:11:28 +09:00
ketank-new 1a3858a769 [SPARK-26985][CORE] Fix "access only some column of the all of columns " for big endian architecture BUG
continuation to https://github.com/apache/spark/pull/24788

## What changes were proposed in this pull request?

Changes are related to BIG ENDIAN system
This changes are done to

identify s390x platform.
use byteorder to BIG_ENDIAN for big endian systems
changes for 2 are done in access functions putFloats() and putDouble()

## How was this patch tested?

Changes have been tested to build successfully on s390x as well x86 platform to make sure build is successful.

Closes #24861 from ketank-new/ketan_latest_v2.3.2.

Authored-by: ketank-new <ketan22584@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 08:24:10 -05:00
HyukjinKwon ac61f7d018 [SPARK-27893][SQL][PYTHON][FOLLOW-UP] Allow Scalar Pandas and Python UDFs can be tested with Scala test base
## What changes were proposed in this pull request?

After this PR, we can test Pandas and Python UDF as below **in Scala side**:

```scala
import IntegratedUDFTestUtils._
val pandasTestUDF = TestScalarPandasUDF("udf")
spark.range(10).select(pandasTestUDF($"id")).show()
```

## How was this patch tested?

Manually tested.

Closes #24945 from HyukjinKwon/SPARK-27893-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-25 12:00:05 +09:00
Gengliang Wang b5e183cdc7 [SPARK-28108][SQL][test-hadoop3.2] Simplify OrcFilters
## What changes were proposed in this pull request?

In #24068, IvanVergiliev fixes the issue that OrcFilters.createBuilder has exponential complexity in the height of the filter tree due to the way the check-and-build pattern is implemented.

Comparing to the approach in #24068, I propose a simple solution for the issue:
1. separate the logic of building a convertible filter tree and the actual SearchArgument builder, since the two procedures are different and their return types are different. Thus the new introduced class `ActionType`,`TrimUnconvertibleFilters` and `BuildSearchArgument`  in #24068 can be dropped. The code is more readable.
2. For most of the leaf nodes, the convertible result is always Some(node), we can abstract it like this PR.
3. The code is actually small changes on the previous code. See https://github.com/apache/spark/pull/24783

## How was this patch tested?
Run the benchmark provided in #24068:
```
val schema = StructType.fromDDL("col INT")
(20 to 30).foreach { width =>
  val whereFilter = (1 to width).map(i => EqualTo("col", i)).reduceLeft(Or)
  val start = System.currentTimeMillis()
  OrcFilters.createFilter(schema, Seq(whereFilter))
  println(s"With $width filters, conversion takes ${System.currentTimeMillis() - start} ms")
}
```
Result:
```
With 20 filters, conversion takes 6 ms
With 21 filters, conversion takes 0 ms
With 22 filters, conversion takes 0 ms
With 23 filters, conversion takes 0 ms
With 24 filters, conversion takes 0 ms
With 25 filters, conversion takes 0 ms
With 26 filters, conversion takes 0 ms
With 27 filters, conversion takes 0 ms
With 28 filters, conversion takes 0 ms
With 29 filters, conversion takes 0 ms
With 30 filters, conversion takes 0 ms
```

Also verified with Unit tests.

Closes #24910 from gengliangwang/refactorOrcFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-24 12:23:52 +08:00
Liang-Chi Hsieh a00774afea [SPARK-28054][SQL] Fix error when insert Hive partitioned table dynamically where partition name is upper case
## What changes were proposed in this pull request?

When we use upper case partition name in Hive table, like:

```
CREATE TABLE src (KEY STRING, VALUE STRING) PARTITIONED BY (DS STRING)
```

Then, `insert into table` query doesn't work
```
INSERT INTO TABLE src PARTITION(ds) SELECT 'k' key, 'v' value, '1' ds
// or
INSERT INTO TABLE src PARTITION(DS) SELECT 'k' KEY, 'v' VALUE, '1' DS
```

```
[info]   org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.Table.ValidationFailureSemanticException: Partition spec {ds=, DS=1} contains non-partition columns;
```

As Hive metastore is not case preserving and keeps partition columns with lower cased names, we lowercase column names in partition spec before passing to Hive client. But we write upper case column names in partition paths.

However, when calling `loadDynamicPartitions` to do `insert into table` for dynamic partition, Hive calculates full path spec for partition paths. So it calculates a partition spec like `{ds=, DS=1}` in above case and fails partition column validation. This patch is proposed to fix the issue by lowercasing the column names in written partition paths for Hive partitioned table.

This fix touchs `saveAsHiveFile` method, which is used in `InsertIntoHiveDirCommand` and `InsertIntoHiveTable` commands. Among them, only `InsertIntoHiveTable` passes `partitionAttributes` parameter. So I think this change only affects `InsertIntoHiveTable` command.

## How was this patch tested?

Added test.

Closes #24886 from viirya/SPARK-28054.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-24 09:44:38 +09:00
Parth Chandra 5a7aa6f4df [SPARK-27100][SQL] Use Array instead of Seq in FilePartition to prevent StackOverflowError
## What changes were proposed in this pull request?

ShuffleMapTask's partition field is a FilePartition and FilePartition's 'files' field is a Stream$cons which is essentially a linked list. It is therefore serialized recursively.
If the number of files in each partition is, say, 10000 files, recursing into a linked list of length 10000 overflows the stack

The problem is only in Bucketed partitions. The corresponding implementation for non Bucketed partitions uses a StreamBuffer. The proposed change applies the same for Bucketed partitions.

## How was this patch tested?

Existing unit tests. Added new unit test. The unit test fails without the patch. Manual testing on dataset used to reproduce the problem.

Closes #24865 from parthchandra/SPARK-27100.

Lead-authored-by: Parth Chandra <parthc@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-06-23 07:47:32 +00:00
Yuming Wang 929d313568 [SPARK-28059][SQL][TEST] Port int4.sql
## What changes were proposed in this pull request?

This PR is to port int4.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/int4.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/int4.out

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28023](https://issues.apache.org/jira/browse/SPARK-28023): Trim the string when cast string type to other types
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027): Add bitwise shift left/right operators

Also, found a bug:
[SPARK-28024](https://issues.apache.org/jira/browse/SPARK-28024): Incorrect value when out of range

Also, found four inconsistent behavior:
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Invalid input syntax for integer: "34.5" at PostgreSQL
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027) Our operator `!` and `!!` has different meanings
[SPARK-28028](https://issues.apache.org/jira/browse/SPARK-28028): Cast numeric to integral type need round
[SPARK-2659](https://issues.apache.org/jira/browse/SPARK-2659): HiveQL: Division operator should always perform fractional division, for example:
```sql
select 1/2;
```

## How was this patch tested?

N/A

Closes #24877 from wangyum/SPARK-28059.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 23:59:30 -07:00
Yuming Wang 0768fad777 [SPARK-28126][SQL] Support TRIM(trimStr FROM str) syntax
## What changes were proposed in this pull request?
[PostgreSQL](7c850320d8/src/test/regress/sql/strings.sql (L624)) support  another trim pattern: `TRIM(trimStr FROM str)`:

Function | Return Type | Description | Example | Result
--- | --- | --- | --- | ---
trim([leading \| trailing \| both] [characters] from string) | text | Remove the longest string containing only characters from characters (a space by default) from the start, end, or both ends (both is the default) of string | trim(both 'xyz' from 'yxTomxx') | Tom

This pr add support this trim pattern. After this pr. We can support all standard syntax except `TRIM(FROM str)` because it conflicts with our Literals:
```sql
Literals of type 'FROM' are currently not supported.(line 1, pos 12)

== SQL ==
SELECT TRIM(FROM ' SPARK SQL ')
```

PostgreSQL, Vertica and MySQL support this pattern. Teradata, Oracle, DB2, SQL Server, Hive and Presto
**PostgreSQL**:
```
postgres=# SELECT substr(version(), 0, 16), trim('xyz' FROM 'yxTomxx');
     substr      | btrim
-----------------+-------
 PostgreSQL 11.3 | Tom
(1 row)
```
**Vertica**:
```
dbadmin=> SELECT version(), trim('xyz' FROM 'yxTomxx');
              version               | btrim
------------------------------------+-------
 Vertica Analytic Database v9.1.1-0 | Tom
(1 row)
```
**MySQL**:
```
mysql> SELECT version(), trim('xyz' FROM 'yxTomxx');
+-----------+----------------------------+
| version() | trim('xyz' FROM 'yxTomxx') |
+-----------+----------------------------+
| 5.7.26    | yxTomxx                    |
+-----------+----------------------------+
1 row in set (0.00 sec)
```

More details:
https://www.postgresql.org/docs/11/functions-string.html

## How was this patch tested?

unit tests

Closes #24924 from wangyum/SPARK-28075-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 23:10:09 -07:00
Yuming Wang 870f972dcc [SPARK-28104][SQL] Implement Spark's own GetColumnsOperation
## What changes were proposed in this pull request?

[SPARK-24196](https://issues.apache.org/jira/browse/SPARK-24196) and [SPARK-24570](https://issues.apache.org/jira/browse/SPARK-24570) implemented Spark's own `GetSchemasOperation` and `GetTablesOperation`. This pr implements Spark's own `GetColumnsOperation`.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/59745367-3a7d6180-92a7-11e9-862d-96bc494c5f00.png)

Closes #24906 from wangyum/SPARK-28104.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 09:15:07 -07:00
Bryan Cutler 5ad1053f3e [SPARK-28128][PYTHON][SQL] Pandas Grouped UDFs skip empty partitions
## What changes were proposed in this pull request?

When running FlatMapGroupsInPandasExec or AggregateInPandasExec the shuffle uses a default number of partitions of 200 in "spark.sql.shuffle.partitions". If the data is small, e.g. in testing, many of the partitions will be empty but are treated just the same.

This PR checks the `mapPartitionsInternal` iterator to be non-empty before calling `ArrowPythonRunner` to start computation on the iterator.

## How was this patch tested?

Existing tests. Ran the following benchmarks a simple example where most partitions are empty:

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.types import *

df = spark.createDataFrame(
     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
     ("id", "v"))

pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP)
def normalize(pdf):
    v = pdf.v
    return pdf.assign(v=(v - v.mean()) / v.std())

df.groupby("id").apply(normalize).count()
```

**Before**
```
In [4]: %timeit df.groupby("id").apply(normalize).count()
1.58 s ± 62.8 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [5]: %timeit df.groupby("id").apply(normalize).count()
1.52 s ± 29.5 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [6]: %timeit df.groupby("id").apply(normalize).count()
1.52 s ± 37.8 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
```

**After this Change**
```
In [2]: %timeit df.groupby("id").apply(normalize).count()
646 ms ± 89.9 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [3]: %timeit df.groupby("id").apply(normalize).count()
408 ms ± 84.6 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [4]: %timeit df.groupby("id").apply(normalize).count()
381 ms ± 29.9 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
```

Closes #24926 from BryanCutler/pyspark-pandas_udf-map-agg-skip-empty-parts-SPARK-28128.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-22 11:20:35 +09:00
Yesheng Ma 54da3bbfb2 [SPARK-28127][SQL] Micro optimization on TreeNode's mapChildren method
## What changes were proposed in this pull request?

The `mapChildren` method in the TreeNode class is commonly used across the whole Spark SQL codebase. In this method, there's a if statement that checks non-empty children. However, there's a cached lazy val `containsChild`, which can avoid unnecessary computation since `containsChild` is used in other methods and therefore constructed anyway.

Benchmark showed that this optimization can improve the whole TPC-DS planning time by 6.8%. There is no regression on any TPC-DS query.

## How was this patch tested?

Existing UTs.

Closes #24925 from yeshengm/treenode-children.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-20 19:45:59 -07:00
sychen d9697fedf5 [SPARK-28012][SQL] Hive UDF supports struct type foldable expression
## What changes were proposed in this pull request?

Currently using hive udf, the parameter is struct type, there will be an exception thrown.

No handler for Hive UDF 'xxxUDF': java.lang.RuntimeException: Hive doesn't support the constant type [StructType(StructField(name,StringType,true), StructField(value,DecimalType(3,1),true))]

## How was this patch tested?
added new UT

Closes #24846 from cxzl25/hive_udf_literal_struct_type.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-20 14:36:01 +09:00