Commit graph

8532 commits

Author SHA1 Message Date
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
Yuming Wang 4968f87168 [SPARK-23263][TEST] CTAS should update stat if autoUpdate statistics is enabled
## What changes were proposed in this pull request?
The [SPARK-27403](https://issues.apache.org/jira/browse/SPARK-27403) fixed CTAS cannot update statistics even if `spark.sql.statistics.size.autoUpdate.enabled` is enabled, as mentioned in [SPARK-23263](https://issues.apache.org/jira/browse/SPARK-23263).

This pr adds tests for that fix.

## How was this patch tested?

N/A

Closes #20430 from wangyum/SPARK-23263.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-20 14:19:10 +09:00
Gengliang Wang f5107614d6 [SPARK-28089][SQL] File source v2: support reading output of file streaming Sink
## What changes were proposed in this pull request?

File source V1 supports reading output of FileStreamSink as batch. https://github.com/apache/spark/pull/11897
We should support this in file source V2 as well. When reading with paths, we first check if there is metadata log of FileStreamSink. If yes, we use `MetadataLogFileIndex` for listing files; Otherwise, we use `InMemoryFileIndex`.

## How was this patch tested?

Unit test

Closes #24900 from gengliangwang/FileStreamV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-20 12:57:13 +08:00
WeichenXu b276788d57 [SPARK-27990][SQL][ML] Provide a way to recursively load data from datasource
## What changes were proposed in this pull request?

Provide a way to recursively load data from datasource.
I add a "recursiveFileLookup" option.

When "recursiveFileLookup" option turn on, then partition inferring is turned off and all files from the directory will be loaded recursively.

If some datasource explicitly specify the partitionSpec, then if user turn on "recursive" option, then exception will be thrown.

## How was this patch tested?

Unit tests.

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

Closes #24830 from WeichenXu123/recursive_ds.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-20 12:43:01 +08:00
Josh Rosen fc65e0fe2c [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes
## What changes were proposed in this pull request?

This PR significantly improves the performance of `UTF8String.replace()` by performing direct replacement over UTF8 bytes instead of decoding those bytes into Java Strings.

In cases where the search string is not found (i.e. no replacements are performed, a case which I expect to be common) this new implementation performs no object allocation or memory copying.

My implementation is modeled after `commons-lang3`'s `StringUtils.replace()` method. As part of my implementation, I needed a StringBuilder / resizable buffer, so I moved `UTF8StringBuilder` from the `catalyst` package to `unsafe`.

## How was this patch tested?

Copied tests from `StringExpressionSuite` to `UTF8StringSuite` and added a couple of new cases.

To evaluate performance, I did some quick local benchmarking by running the following code in `spark-shell` (with Java 1.8.0_191):

```scala
import org.apache.spark.unsafe.types.UTF8String

def benchmark(text: String, search: String, replace: String) {
  val utf8Text = UTF8String.fromString(text)
  val utf8Search = UTF8String.fromString(search)
  val utf8Replace = UTF8String.fromString(replace)

  val start = System.currentTimeMillis
  var i = 0
  while (i < 1000 * 1000 * 100) {
    utf8Text.replace(utf8Search, utf8Replace)
    i += 1
  }
  val end = System.currentTimeMillis

  println(end - start)
}

benchmark("ABCDEFGH", "DEF", "ZZZZ")  // replacement occurs
benchmark("ABCDEFGH", "Z", "")  // no replacement occurs
```

On my laptop this took ~54 / ~40 seconds seconds before this patch's changes and ~6.5 / ~3.8 seconds afterwards.

Closes #24707 from JoshRosen/faster-string-replace.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-06-19 15:21:26 -07:00
Yuming Wang fe5145ede2 [SPARK-28109][SQL] Fix TRIM(type trimStr FROM str) returns incorrect value
## What changes were proposed in this pull request?

[SPARK-28093](https://issues.apache.org/jira/browse/SPARK-28093) fixed `TRIM/LTRIM/RTRIM('str', 'trimStr')` returns an incorrect value, but that fix introduced a new bug, `TRIM(type trimStr FROM str)` returns an incorrect value. This pr fix this issue.

## How was this patch tested?

unit tests and manual tests:
Before this PR:
```sql
spark-sql> SELECT trim('yxTomxx', 'xyz'), trim(BOTH 'xyz' FROM 'yxTomxx');
Tom	z
spark-sql> SELECT trim('xxxbarxxx', 'x'), trim(BOTH 'x' FROM 'xxxbarxxx');
bar
spark-sql> SELECT ltrim('zzzytest', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytest');
test	xyz
spark-sql> SELECT ltrim('zzzytestxyz', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytestxyz');
testxyz
spark-sql> SELECT ltrim('xyxXxyLAST WORD', 'xy'), trim(LEADING 'xy' FROM 'xyxXxyLAST WORD');
XxyLAST WORD
spark-sql> SELECT rtrim('testxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'testxxzx');
test	xy
spark-sql> SELECT rtrim('xyztestxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'xyztestxxzx');
xyztest
spark-sql> SELECT rtrim('TURNERyxXxy', 'xy'), trim(TRAILING 'xy' FROM 'TURNERyxXxy');
TURNERyxX
```
After this PR:
```sql
spark-sql> SELECT trim('yxTomxx', 'xyz'), trim(BOTH 'xyz' FROM 'yxTomxx');
Tom     Tom
spark-sql> SELECT trim('xxxbarxxx', 'x'), trim(BOTH 'x' FROM 'xxxbarxxx');
bar     bar
spark-sql> SELECT ltrim('zzzytest', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytest');
test    test
spark-sql> SELECT ltrim('zzzytestxyz', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytestxyz');
testxyz testxyz
spark-sql> SELECT ltrim('xyxXxyLAST WORD', 'xy'), trim(LEADING 'xy' FROM 'xyxXxyLAST WORD');
XxyLAST WORD    XxyLAST WORD
spark-sql> SELECT rtrim('testxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'testxxzx');
test    test
spark-sql> SELECT rtrim('xyztestxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'xyztestxxzx');
xyztest xyztest
spark-sql> SELECT rtrim('TURNERyxXxy', 'xy'), trim(TRAILING 'xy' FROM 'TURNERyxXxy');
TURNERyxX       TURNERyxX
```
And PostgreSQL:
```sql
postgres=# SELECT trim('yxTomxx', 'xyz'), trim(BOTH 'xyz' FROM 'yxTomxx');
 btrim | btrim
-------+-------
 Tom   | Tom
(1 row)

postgres=# SELECT trim('xxxbarxxx', 'x'), trim(BOTH 'x' FROM 'xxxbarxxx');
 btrim | btrim
-------+-------
 bar   | bar
(1 row)

postgres=# SELECT ltrim('zzzytest', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytest');
 ltrim | ltrim
-------+-------
 test  | test
(1 row)

postgres=# SELECT ltrim('zzzytestxyz', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytestxyz');
  ltrim  |  ltrim
---------+---------
 testxyz | testxyz
(1 row)

postgres=# SELECT ltrim('xyxXxyLAST WORD', 'xy'), trim(LEADING 'xy' FROM 'xyxXxyLAST WORD');
    ltrim     |    ltrim
--------------+--------------
 XxyLAST WORD | XxyLAST WORD
(1 row)

postgres=# SELECT rtrim('testxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'testxxzx');
 rtrim | rtrim
-------+-------
 test  | test
(1 row)

postgres=# SELECT rtrim('xyztestxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'xyztestxxzx');
  rtrim  |  rtrim
---------+---------
 xyztest | xyztest
(1 row)

postgres=# SELECT rtrim('TURNERyxXxy', 'xy'), trim(TRAILING 'xy' FROM 'TURNERyxXxy');
   rtrim   |   rtrim
-----------+-----------
 TURNERyxX | TURNERyxX
(1 row)
```

Closes #24911 from wangyum/SPARK-28109.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-19 12:47:18 -07:00
Yesheng Ma 7b7f16f2a7 [SPARK-27890][SQL] Improve SQL parser error message for character-only identifier with hyphens except those in expressions
## What changes were proposed in this pull request?

Current SQL parser's error message for hyphen-connected identifiers without surrounding backquotes(e.g. hyphen-table) is confusing for end users. A possible approach to tackle this is to explicitly capture these wrong usages in the SQL parser. In this way, the end users can fix these errors more quickly.

For example, for a simple query such as `SELECT * FROM test-table`, the original error message is
```
Error in SQL statement: ParseException:
mismatched input '-' expecting <EOF>(line 1, pos 18)
```
which can be confusing in a large query.

After the fix, the error message is:
```
Error in query:
Possibly unquoted identifier test-table detected. Please consider quoting it with back-quotes as `test-table`(line 1, pos 14)

== SQL ==
SELECT * FROM test-table
--------------^^^
```
which is easier for end users to identify the issue and fix.

We safely augmented the current grammar rule to explicitly capture these error cases. The error handling logic is implemented in the SQL parsing listener `PostProcessor`.

However, note that for cases such as `a - my-func(b)`, the parser can't actually tell whether this should be ``a -`my-func`(b) `` or `a - my - func(b)`. Therefore for these cases, we leave the parser as is. Also, in this patch we only provide better error messages for character-only identifiers.

## How was this patch tested?
Adding new unit tests.

Closes #24749 from yeshengm/hyphen-ident.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-18 21:51:15 -07:00
Yesheng Ma 15de6d0500 [SPARK-28096][SQL] Convert defs to lazy vals to avoid expensive reference computation in QueryPlan and Expression
## What changes were proposed in this pull request?

The original `references` and `validConstraints` implementations in a few `QueryPlan` and `Expression` classes are methods, which means unnecessary re-computation can happen at times. This PR resolves this problem by making these method `lazy val`s.

As shown in the following chart, the planning time(without cost-based optimization) was dramatically reduced after this optimization.
- The average planning time of TPC-DS queries was reduced by 19.63%.
- The planning time of the most time-consuming TPC-DS query (q64) was reduced by 43.03%.
- The running time for rule-based reordering joins(not cost-based join reordering) optimization, which are common in real-world OLAP queries,  was largely reduced.

![chart](https://user-images.githubusercontent.com/12269969/59721493-536a1200-91d6-11e9-9bfb-d7cb1e841a86.png)

Detailed stats are listed in the following spreadsheet (we warmed up the queries 5 iterations and then took average of the next 5 iterations).
[Lazy val benchmark.xlsx](https://github.com/apache/spark/files/3303530/Lazy.val.benchmark.xlsx)

## How was this patch tested?

Existing UTs.

Closes #24866 from yeshengm/plannode-micro-opt.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-18 21:13:50 -07:00
Ivan Vergiliev a5dcb82b5a [SPARK-27105][SQL] Optimize away exponential complexity in ORC predicate conversion
## What changes were proposed in this pull request?

`OrcFilters.createBuilder` has exponential complexity in the height of the filter tree due to the way the check-and-build pattern is implemented. We've hit this in production by passing a `Column` filter to Spark directly, with a job taking multiple hours for a simple set of ~30 filters. This PR changes the checking logic so that the conversion has linear complexity in the size of the tree instead of exponential in its height.

Right now, due to the way ORC `SearchArgument` works, the code is forced to do two separate phases when converting a given Spark filter to an ORC filter:
1. Check if the filter is convertible.
2. Only if the check in 1. succeeds, perform the actual conversion into the resulting ORC filter.

However, there's one detail which is the culprit in the exponential complexity: phases 1. and 2. are both done using the exact same method. The resulting exponential complexity is easiest to see in the `NOT` case - consider the following code:

```
val f1 = col("id") === lit(5)
val f2 = !f1
val f3 = !f2
val f4 = !f3
val f5 = !f4
```

Now, when we run `createBuilder` on `f5`, we get the following behaviour:
1. call `createBuilder(f4)` to check if the child `f4` is convertible
2. call `createBuilder(f4)` to actually convert it

This seems fine when looking at a single level, but what actually ends up happening is:
- `createBuilder(f3)` will then recursively be called 4 times - 2 times in step 1., and two times in step 2.
- `createBuilder(f2)` will be called 8 times - 4 times in each top-level step, 2 times in each sub-step.
- `createBuilder(f1)` will be called 16 times.

As a result, having a tree of height > 30 leads to billions of calls to `createBuilder`, heap allocations, and so on and can take multiple hours.

The way this PR solves this problem is by separating the `check` and `convert` functionalities into separate functions. This way, the call to `createBuilder` on `f5` above would look like this:
1. call `isConvertible(f4)` to check if the child `f4` is convertible - amortized constant complexity
2. call `createBuilder(f4)` to actually convert it - linear complexity in the size of the subtree.

This way, we get an overall complexity that's linear in the size of the filter tree, allowing us to convert tree with 10s of thousands of nodes in milliseconds.

The reason this split (`check` and `build`) is possible is that the checking never actually depends on the actual building of the filter. The `check` part of `createBuilder` depends mainly on:
- `isSearchableType` for leaf nodes, and
- `check`-ing the child filters for composite nodes like NOT, AND and OR.
Situations like the `SearchArgumentBuilder` throwing an exception while building the resulting ORC filter are not handled right now - they just get thrown out of the class, and this change preserves this behaviour.

This PR extracts this part of the code to a separate class which allows the conversion to make very efficient checks to confirm that a given child is convertible before actually converting it.

Results:
Before:
- converting a skewed tree with a height of ~35 took about 6-7 hours.
- converting a skewed tree with hundreds or thousands of nodes would be completely impossible.

Now:
- filtering against a skewed tree with a height of 1500 in the benchmark suite finishes in less than 10 seconds.

## Steps to reproduce
```scala
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")
}
```

### Before this PR
```
With 20 filters, conversion takes 363 ms
With 21 filters, conversion takes 496 ms
With 22 filters, conversion takes 939 ms
With 23 filters, conversion takes 1871 ms
With 24 filters, conversion takes 3756 ms
With 25 filters, conversion takes 7452 ms
With 26 filters, conversion takes 14978 ms
With 27 filters, conversion takes 30519 ms
With 28 filters, conversion takes 60361 ms // 1 minute
With 29 filters, conversion takes 126575 ms // 2 minutes 6 seconds
With 30 filters, conversion takes 257369 ms // 4 minutes 17 seconds
```

### After this PR
```
With 20 filters, conversion takes 12 ms
With 21 filters, conversion takes 0 ms
With 22 filters, conversion takes 1 ms
With 23 filters, conversion takes 0 ms
With 24 filters, conversion takes 1 ms
With 25 filters, conversion takes 1 ms
With 26 filters, conversion takes 0 ms
With 27 filters, conversion takes 1 ms
With 28 filters, conversion takes 0 ms
With 29 filters, conversion takes 1 ms
With 30 filters, conversion takes 0 ms
```

## How was this patch tested?

There are no changes in behaviour, and the existing tests pass. Added new benchmarks that expose the problematic behaviour and they finish quickly with the changes applied.

Closes #24068 from IvanVergiliev/optimize-orc-filters.

Authored-by: Ivan Vergiliev <ivan.vergiliev@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-19 10:44:58 +08:00
Yuming Wang 2e3ae97668 [SPARK-28039][SQL][TEST] Port float4.sql
## What changes were proposed in this pull request?

This PR is to port float4.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/float4.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/float4.out

When porting the test cases, found three PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28060](https://issues.apache.org/jira/browse/SPARK-28060): Float type can not accept some special inputs
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027): Spark SQL does not support prefix operator ``
[SPARK-28061](https://issues.apache.org/jira/browse/SPARK-28061): Support for converting float to binary format

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): Spark SQL insert there 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

## How was this patch tested?

N/A

Closes #24887 from wangyum/SPARK-28039.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-18 16:22:30 -07:00
Yuming Wang c7f0301477 [SPARK-28088][SQL] Enhance LPAD/RPAD function
## What changes were proposed in this pull request?

This pr enhances `LPAD`/`RPAD` function to make `pad` parameter optional.

PostgreSQL, Vertica, Teradata, Oracle and DB2 support make `pad` parameter optional. MySQL, Hive and Presto does not support make `pad` parameter optional. SQL Server does not have `lapd`/`rpad` function.
**PostgreSQL**:
```
postgres=# select substr(version(), 0, 16), lpad('hi', 5), rpad('hi', 5);
     substr      | lpad  | rpad
-----------------+-------+-------
 PostgreSQL 11.3 |    hi | hi
(1 row)
```
**Vertica**:
```
dbadmin=> select version(), lpad('hi', 5), rpad('hi', 5);
              version               | lpad  | rpad
------------------------------------+-------+-------
 Vertica Analytic Database v9.1.1-0 |    hi | hi
(1 row)
```
**Teradata**:
![image](https://user-images.githubusercontent.com/5399861/59656550-89a49300-91d0-11e9-9f26-ed554f49ea34.png)
**Oracle**:
![image](https://user-images.githubusercontent.com/5399861/59656591-a9d45200-91d0-11e9-8b0e-3e1f75983099.png)
**DB2**:
![image](https://user-images.githubusercontent.com/5399861/59656468-3e8a8000-91d0-11e9-8826-0d854ed7f397.png)

More details:
https://www.postgresql.org/docs/11/functions-string.html
https://docs.teradata.com/reader/kmuOwjp1zEYg98JsB8fu_A/e5w8LujIQDlVmRSww2E27A

## How was this patch tested?

unit tests

Closes #24899 from wangyum/SPARK-28088.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-18 14:08:18 -07:00
Yuming Wang bef5d9d6c3 [SPARK-28093][SQL] Fix TRIM/LTRIM/RTRIM function parameter order issue
## What changes were proposed in this pull request?

This pr fix `TRIM`/`LTRIM`/`RTRIM` function parameter order issue, otherwise:

```sql
spark-sql> SELECT trim('yxTomxx', 'xyz'), trim('xxxbarxxx', 'x');
z
spark-sql> SELECT ltrim('zzzytest', 'xyz'), ltrim('xyxXxyLAST WORD', 'xy');
xyz
spark-sql> SELECT rtrim('testxxzx', 'xyz'), rtrim('TURNERyxXxy', 'xy');
xy
spark-sql>
```

```sql
postgres=# SELECT trim('yxTomxx', 'xyz'), trim('xxxbarxxx', 'x');
 btrim | btrim
-------+-------
 Tom   | bar
(1 row)

postgres=# SELECT ltrim('zzzytest', 'xyz'), ltrim('xyxXxyLAST WORD', 'xy');
 ltrim |    ltrim
-------+--------------
 test  | XxyLAST WORD
(1 row)

postgres=# SELECT rtrim('testxxzx', 'xyz'), rtrim('TURNERyxXxy', 'xy');
 rtrim |   rtrim
-------+-----------
 test  | TURNERyxX
(1 row)
```

## How was this patch tested?

unit tests

Closes #24902 from wangyum/SPARK-28093.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-18 13:28:29 -07:00
maryannxue 1ada36b571 [SPARK-27783][SQL] Add customizable hint error handler
## What changes were proposed in this pull request?

Added an interface for handling hint errors, with a default implementation class that logs warnings in the callbacks.

## How was this patch tested?

Passed existing tests.

Closes #24653 from maryannxue/hint-handler.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-18 12:33:32 -07:00
Dongjoon Hyun ed280c23ca [SPARK-28072][SQL] Fix IncompatibleClassChangeError in FromUnixTime codegen on JDK9+
## What changes were proposed in this pull request?

With JDK9+, the generate **bytecode** of `FromUnixTime` raise `java.lang.IncompatibleClassChangeError` due to [JDK-8145148](https://bugs.openjdk.java.net/browse/JDK-8145148) . This is a blocker in [Apache Spark JDK11 Jenkins job](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing/). Locally, this is reproducible by the following unit test suite with JDK9+.
```
$ build/sbt "catalyst/testOnly *.DateExpressionsSuite"
...
[info] org.apache.spark.sql.catalyst.expressions.DateExpressionsSuite *** ABORTED *** (23 seconds, 75 milliseconds)
[info]   java.lang.IncompatibleClassChangeError: Method org.apache.spark.sql.catalyst.util.TimestampFormatter.apply(Ljava/lang/String;Ljava/time/ZoneId;Ljava/util/Locale;)Lorg/apache/spark/sql/catalyst/util/TimestampFormatter; must be InterfaceMeth
```

This bytecode issue is generated by `Janino` , so we replace `.apply` to `.MODULE$$.apply` and adds test coverage for similar codes.

## How was this patch tested?

Manually with the existing UTs by doing the following with JDK9+.
```
build/sbt "catalyst/testOnly *.DateExpressionsSuite"
```

Actually, this is the last JDK11 error in `catalyst` module. So, we can verify with the following, too.
```
$ build/sbt "project catalyst" test
...
[info] Total number of tests run: 3552
[info] Suites: completed 210, aborted 0
[info] Tests: succeeded 3552, failed 0, canceled 0, ignored 2, pending 0
[info] All tests passed.
[info] Passed: Total 3583, Failed 0, Errors 0, Passed 3583, Ignored 2
[success] Total time: 294 s, completed Jun 16, 2019, 10:15:08 PM
```

Closes #24889 from dongjoon-hyun/SPARK-28072.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-18 00:08:37 -07:00
Liang-Chi Hsieh b7bdc3111e [SPARK-28058][DOC] Add a note to doc of mode of CSV for column pruning
## What changes were proposed in this pull request?

When using `DROPMALFORMED` mode, corrupted records aren't dropped if malformed columns aren't read. This behavior is due to CSV parser column pruning. Current doc of `DROPMALFORMED` doesn't mention the effect of column pruning. Users will be confused by the fact that `DROPMALFORMED` mode doesn't work as expected.

Column pruning also affects other modes. This is a doc improvement to add a note to doc of `mode` to explain it.

## How was this patch tested?

N/A. This is just doc change.

Closes #24894 from viirya/SPARK-28058.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-18 13:48:32 +09:00
Yuming Wang ab6bb8fc1c [SPARK-28075][SQL] Enhance TRIM function
## What changes were proposed in this pull request?

The `TRIM` function accept these patterns:
```sql
TRIM(str)
TRIM(trimStr, str)
TRIM(BOTH trimStr FROM str)
TRIM(LEADING trimStr FROM str)
TRIM(TRAILING trimStr FROM str)
```
This pr add support other three patterns:
```sql
TRIM(BOTH FROM str)
TRIM(LEADING FROM str)
TRIM(TRAILING FROM str)
```

PostgreSQL, Vertica, MySQL, Teradata, Oracle and DB2 support these patterns. Hive, Presto and SQL Server does not support this feature.

**PostgreSQL**:
```sql
postgres=# select substr(version(), 0, 16), trim(BOTH from '    SparkSQL   '), trim(LEADING FROM '    SparkSQL   '), trim(TRAILING FROM '    SparkSQL   ');
     substr      |  btrim   |    ltrim    |    rtrim
-----------------+----------+-------------+--------------
 PostgreSQL 11.3 | SparkSQL | SparkSQL    |     SparkSQL
(1 row)
```
**Vertica**:
```
dbadmin=> select version(), trim(BOTH from '    SparkSQL   '), trim(LEADING FROM '    SparkSQL   '), trim(TRAILING FROM '    SparkSQL   ');
              version               |  btrim   |    ltrim    |    rtrim
------------------------------------+----------+-------------+--------------
 Vertica Analytic Database v9.1.1-0 | SparkSQL | SparkSQL    |     SparkSQL
(1 row)
```
**MySQL**:
```
mysql> select version(), trim(BOTH from '    SparkSQL   '), trim(LEADING FROM '    SparkSQL   '), trim(TRAILING FROM '    SparkSQL   ');
+-----------+-----------------------------------+--------------------------------------+---------------------------------------+
| version() | trim(BOTH from '    SparkSQL   ') | trim(LEADING FROM '    SparkSQL   ') | trim(TRAILING FROM '    SparkSQL   ') |
+-----------+-----------------------------------+--------------------------------------+---------------------------------------+
| 5.7.26    | SparkSQL                          | SparkSQL                             |     SparkSQL                          |
+-----------+-----------------------------------+--------------------------------------+---------------------------------------+
1 row in set (0.01 sec)
```
**Teradata**:
![image](https://user-images.githubusercontent.com/5399861/59587081-070bcd00-9117-11e9-8534-df547860b585.png)
**Oracle**:
![image](https://user-images.githubusercontent.com/5399861/59587003-cf048a00-9116-11e9-839e-90da9e5183e0.png)
**DB2**:
![image](https://user-images.githubusercontent.com/5399861/59587801-af6e6100-9118-11e9-80be-ee1f6bbbeceb.png)

## How was this patch tested?

unit tests

Closes #24891 from wangyum/SPARK-28075.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-18 12:26:10 +08:00
Dongjoon Hyun d6a479b1f8 [SPARK-28063][SQL] Replace deprecated .newInstance() in DSv2 Catalogs
## What changes were proposed in this pull request?

This PR aims to replace deprecated `.newInstance()` in DSv2 `Catalogs` and distinguish the plugin class errors more. According to the JDK11 build log, there is no other new instance.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing/978/consoleFull

SPARK-25984 removes all instances of the deprecated `.newInstance()` usages at Nov 10, 2018, but this was added at SPARK-24252 on March 8, 2019.

## How was this patch tested?

Pass the Jenkins with the updated test case.

Closes #24882 from dongjoon-hyun/SPARK-28063.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-16 19:58:02 -07:00
Takuya UESHIN 5ae1a6bf0d [SPARK-28052][SQL] Make ArrayExists follow the three-valued boolean logic.
## What changes were proposed in this pull request?

Currently `ArrayExists` always returns boolean values (if the arguments are not `null`), but it should follow the three-valued boolean logic:

- `true` if the predicate holds at least one `true`
- otherwise, `null` if the predicate holds `null`
- otherwise, `false`

This behavior change is made to match Postgres' equivalent function `ANY/SOME (array)`'s behavior: https://www.postgresql.org/docs/9.6/functions-comparisons.html#AEN21174

## How was this patch tested?

Modified tests and existing tests.

Closes #24873 from ueshin/issues/SPARK-28052/fix_exists.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-15 10:48:06 -07:00
WeichenXu 6d441dcdc6 [SPARK-26412][PYSPARK][SQL] Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series
## What changes were proposed in this pull request?

Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series.
Note the UDF input args will be always one iterator:
* if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch)
* if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example:
```
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def the_udf(iterator):
    for col1_batch, col2_batch in iterator:
        yield col1_batch + col2_batch

df.select(the_udf("col1", "col2"))
```
The udf above will add col1 and col2.

I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review.
We can test several typical cases:

```
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import udf
from pyspark.taskcontext import TaskContext

df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"])

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi1: do init stuff, partitionId=" + str(pid))
    for batch in it:
        yield batch + 100
    print("DBG: fi1: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi2(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi2: do init stuff, partitionId=" + str(pid))
    for batch in it:
        yield batch + 10000
    print("DBG: fi2: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi3(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi3: do init stuff, partitionId=" + str(pid))
    for x, y in it:
        yield x + y * 10 + 100000
    print("DBG: fi3: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 1000

udf("int")
def fu1(x):
    return x + 10

# test select "pandas iter udf/pandas udf/sql udf" expressions at the same time.
# Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan,
# and `fu1("a")`, `fp1("a")` will generate another two separate plans.
df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show()

# test chain two pandas iter udf together
# Note this case `fi2(fi1("a"))` will generate only one plan
# Also note the init stuff/close stuff call order will be like:
# (debug output following)
#     DBG: fi2: do init stuff, partitionId=0
#     DBG: fi1: do init stuff, partitionId=0
#     DBG: fi1: do close stuff, partitionId=0
#     DBG: fi2: do close stuff, partitionId=0
df.select(fi2(fi1("a"))).show()

# test more complex chain
# Note this case `fi1("a"), fi2("a")` will generate one plan,
# and `fi3(fi1_output, fi2_output)` will generate another plan
df.select(fi3(fi1("a"), fi2("a"))).show()
```

## How was this patch tested?

To be added.

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

Closes #24643 from WeichenXu123/pandas_udf_iter.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-15 08:29:20 -07:00
HyukjinKwon 26998b86c1 [SPARK-27870][SQL][PYTHON] Add a runtime buffer size configuration for Pandas UDFs
## What changes were proposed in this pull request?

This PR is an alternative approach for #24734.

This PR fixes two things:

1. Respects `spark.buffer.size` in Python workers.
2. Adds a runtime buffer size configuration for Pandas UDFs, `spark.sql.pandas.udf.buffer.size` (which falls back to `spark.buffer.size`.

## How was this patch tested?

Manually tested:

```python
import time
from pyspark.sql.functions import *

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```

```
consume time: 62.68265891075134
```

```python
import time
from pyspark.sql.functions import *

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
spark.conf.set('spark.sql.pandas.udf.buffer.size', '4')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```

```
consume time: 34.00594782829285
```

Closes #24826 from HyukjinKwon/SPARK-27870.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-15 20:56:22 +09:00
Gengliang Wang 23ebd389b5 [SPARK-27418][SQL] Migrate Parquet to File Data Source V2
## What changes were proposed in this pull request?

 Migrate Parquet to File Data Source V2

## How was this patch tested?

Unit test

Closes #24327 from gengliangwang/parquetV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-15 20:52:50 +09:00
maryannxue c79f471d04 [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
## What changes were proposed in this pull request?

Implemented a new SparkPlan that executes the query adaptively. It splits the query plan into independent stages and executes them in order according to their dependencies. The query stage materializes its output at the end. When one stage completes, the data statistics of the materialized output will be used to optimize the remainder of the query.

The adaptive mode is off by default, when turned on, user can see "AdaptiveSparkPlan" as the top node of a query or sub-query. The inner plan of "AdaptiveSparkPlan" is subject to change during query execution but becomes final once the execution is complete. Whether the inner plan is final is included in the EXPLAIN string. Below is an example of the EXPLAIN plan before and after execution:

Query:
```
SELECT * FROM testData JOIN testData2 ON key = a WHERE value = '1'
```

Before execution:
```
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- SortMergeJoin [key#13], [a#23], Inner
   :- Sort [key#13 ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(key#13, 5)
   :     +- Filter (isnotnull(value#14) AND (value#14 = 1))
   :        +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false) AS value#14]
   :           +- Scan[obj#12]
   +- Sort [a#23 ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(a#23, 5)
         +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24]
            +- Scan[obj#22]
```

After execution:
```
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=true)
+- *(1) BroadcastHashJoin [key#13], [a#23], Inner, BuildLeft
   :- BroadcastQueryStage 2
   :  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   :     +- ShuffleQueryStage 0
   :        +- Exchange hashpartitioning(key#13, 5)
   :           +- *(1) Filter (isnotnull(value#14) AND (value#14 = 1))
   :              +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false) AS value#14]
   :                 +- Scan[obj#12]
   +- ShuffleQueryStage 1
      +- Exchange hashpartitioning(a#23, 5)
         +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24]
            +- Scan[obj#22]
```

Credit also goes to carsonwang and cloud-fan

## How was this patch tested?

Added new UT.

Closes #24706 from maryannxue/aqe.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: herman <herman@databricks.com>
2019-06-15 11:27:15 +02:00
Peter Toth 9e6666bde1 [SPARK-28002][SQL] Support WITH clause column aliases
## What changes were proposed in this pull request?

This PR adds support of column aliasing in a CTE so this query becomes valid:
```
WITH t(x) AS (SELECT 1)
SELECT * FROM t WHERE x = 1
```
## How was this patch tested?

Added new UTs.

Closes #24842 from peter-toth/SPARK-28002.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-14 20:47:11 -07:00
Jungtaek Lim (HeartSaVioR) bd0a04baab [SPARK-26949][SS] Prevent 'purge' to remove needed batch files in CompactibleFileStreamLog
## What changes were proposed in this pull request?

This patch proposes making `purge` in `CompactibleFileStreamLog` to throw `UnsupportedOperationException` to prevent purging necessary batch files, as well as adding javadoc to document its behavior. Actually it would only break when latest compaction batch is requested to be purged, but caller wouldn't be aware of this so safer to just prevent it.

## How was this patch tested?

Added UT.

Closes #23850 from HeartSaVioR/SPARK-26949.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-14 20:34:18 -07:00
maryannxue d1951aa23b [SPARK-28057][SQL] Add method clone in catalyst TreeNode
## What changes were proposed in this pull request?

Implemented the `clone` method for `TreeNode` based on `mapChildren`.

## How was this patch tested?

Added new UT.

Closes #24876 from maryannxue/treenode-clone.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: herman <herman@databricks.com>
2019-06-15 00:40:55 +02:00
Liang-Chi Hsieh c0297dedd8 [MINOR][PYSPARK][SQL][DOC] Fix rowsBetween doc in Window
## What changes were proposed in this pull request?

I suspect that the doc of `rowsBetween` methods in Scala and PySpark looks wrong.
Because:

```scala
scala> val df = Seq((1, "a"), (2, "a"), (3, "a"), (4, "a"), (5, "a"), (6, "a")).toDF("id", "category")
df: org.apache.spark.sql.DataFrame = [id: int, category: string]

scala> val byCategoryOrderedById = Window.partitionBy('category).orderBy('id).rowsBetween(-1, 2)
byCategoryOrderedById: org.apache.spark.sql.expressions.WindowSpec = org.apache.spark.sql.expressions.WindowSpec7f04de97

scala> df.withColumn("sum", sum('id) over byCategoryOrderedById).show()
+---+--------+---+
| id|category|sum|
+---+--------+---+
|  1|       a|  6|              # sum from index 0 to (0 + 2): 1 + 2 + 3 = 6
|  2|       a| 10|              # sum from index (1 - 1) to (1 + 2): 1 + 2 + 3 + 4 = 10
|  3|       a| 14|
|  4|       a| 18|
|  5|       a| 15|
|  6|       a| 11|
+---+--------+---+
```

So the frame (-1, 2) for row with index 5, as described in the doc, should range from index 4 to index 7.

## How was this patch tested?

N/A, just doc change.

Closes #24864 from viirya/window-spec-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-14 09:56:37 +09:00
Zhu, Lipeng 5700c39c89 [SPARK-27578][SQL] Support INTERVAL ... HOUR TO SECOND syntax
## What changes were proposed in this pull request?

Currently, SparkSQL can support interval format like this.
```sql
SELECT INTERVAL '0 23:59:59.155' DAY TO SECOND
 ```

Like Presto/Teradata, this PR aims to support grammar like below.
```sql
SELECT INTERVAL '23:59:59.155' HOUR TO SECOND
```

Although we can add a new function for this pattern, we had better extend the existing code to handle a missing day case. So, the following is also supported.
```sql
SELECT INTERVAL '23:59:59.155' DAY TO SECOND
SELECT INTERVAL '1 23:59:59.155' HOUR TO SECOND
```
Currently Vertica/Teradata/Postgresql/SQL Server have fully support of below interval functions.
- interval ... year to month
- interval ... day to hour
- interval ... day to minute
- interval ... day to second
- interval ... hour to minute
- interval ... hour to second
- interval ... minute to second

https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Literals/interval-qualifier.htm
df1a699e5b/src/test/regress/sql/interval.sql (L180-L203)
https://docs.teradata.com/reader/S0Fw2AVH8ff3MDA0wDOHlQ/KdCtT3pYFo~_enc8~kGKVw
https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/interval-literals?view=sql-server-2017

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24472 from lipzhu/SPARK-27578.

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-06-13 10:12:55 -07:00
John Zhuge abe370f971 [SPARK-27322][SQL] DataSourceV2 table relation
## What changes were proposed in this pull request?

Support multi-catalog in the following SELECT code paths:

- SELECT * FROM catalog.db.tbl
- TABLE catalog.db.tbl
- JOIN or UNION tables from different catalogs
- SparkSession.table("catalog.db.tbl")
- CTE relation
- View text

## How was this patch tested?

New unit tests.
All existing unit tests in catalyst and sql core.

Closes #24741 from jzhuge/SPARK-27322-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-13 13:48:40 +08:00
Xiangrui Meng 4f4829b4ae [SPARK-28030][SQL] convert filePath to URI in binary file data source
## What changes were proposed in this pull request?

Convert `PartitionedFile.filePath` to URI first in binary file data source. Otherwise Spark will throw a FileNotFound exception because we create `Path` with URL encoded string, instead of wrapping it with URI.

## How was this patch tested?

Unit test.

Closes #24855 from mengxr/SPARK-28030.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-12 13:24:02 -07:00
Liang-Chi Hsieh 2c9597f88f [SPARK-27701][SQL] Extend NestedColumnAliasing to general nested field cases including GetArrayStructField
## What changes were proposed in this pull request?

`NestedColumnAliasing` rule covers `GetStructField` only, currently. It means that some nested field extraction expressions aren't pruned. For example, if only accessing a nested field in an array of struct (`GetArrayStructFields`), this column isn't pruned.

This patch extends the rule to cover general nested field cases, including `GetArrayStructFields`.
## How was this patch tested?

Added tests.

Closes #24599 from viirya/nested-pruning-extract-value.

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-06-11 20:12:53 -07:00
Yesheng Ma 3ddc77d9ac [SPARK-21136][SQL] Disallow FROM-only statements and show better warnings for Hive-style single-from statements
Current Spark SQL parser can have pretty confusing error messages when parsing an incorrect SELECT SQL statement. The proposed fix has the following effect.

BEFORE:
```
spark-sql> SELECT * FROM test WHERE x NOT NULL;
Error in query:
mismatched input 'FROM' expecting {<EOF>, 'CLUSTER', 'DISTRIBUTE', 'EXCEPT', 'GROUP', 'HAVING', 'INTERSECT', 'LATERAL', 'LIMIT', 'ORDER', 'MINUS', 'SORT', 'UNION', 'WHERE', 'WINDOW'}(line 1, pos 9)

== SQL ==
SELECT * FROM test WHERE x NOT NULL
---------^^^
```
where in fact the error message should be hinted to be near `NOT NULL`.

AFTER:
```
spark-sql> SELECT * FROM test WHERE x NOT NULL;
Error in query:
mismatched input 'NOT' expecting {<EOF>, 'AND', 'CLUSTER', 'DISTRIBUTE', 'EXCEPT', 'GROUP', 'HAVING', 'INTERSECT', 'LIMIT', 'OR', 'ORDER', 'MINUS', 'SORT', 'UNION', 'WINDOW'}(line 1, pos 27)

== SQL ==
SELECT * FROM test WHERE x NOT NULL
---------------------------^^^
```

In fact, this problem is brought by some problematic Spark SQL grammar. There are two kinds of SELECT statements that are supported by Hive (and thereby supported in SparkSQL):
* `FROM table SELECT blahblah SELECT blahblah`
* `SELECT blah FROM table`

*Reference* [HiveQL single-from stmt grammar](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g)

It is fine when these two SELECT syntaxes are supported separately. However, since we are currently supporting these two kinds of syntaxes in a single ANTLR rule, this can be problematic and therefore leading to confusing parser errors. This is because when a  SELECT clause was parsed, it can't tell whether the following FROM clause actually belongs to it or is just the beginning of a new `FROM table SELECT *` statement.

## What changes were proposed in this pull request?
1. Modify ANTLR grammar to fix the above-mentioned problem. This fix is important because the previous problematic grammar does affect a lot of real-world queries. Due to the previous problematic and messy grammar, we refactored the grammar related to `querySpecification`.
2. Modify `AstBuilder` to have separate visitors for `SELECT ... FROM ...` and `FROM ... SELECT ...` statements.
3. Drop the `FROM table` statement, which is supported by accident and is actually parsed in the wrong code path. Both Hive and Presto do not support this syntax.

## How was this patch tested?
Existing UTs and new UTs.

Closes #24809 from yeshengm/parser-refactor.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-06-11 18:30:56 -07:00
Yuming Wang 6284ac7088 [SPARK-27934][SQL][TEST] Port case.sql
## What changes were proposed in this pull request?

This PR is to port case.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/case.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/case.out

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

- [SPARK-27930](https://issues.apache.org/jira/browse/SPARK-27930): Add built-in Math Function: RANDOM

## How was this patch tested?

N/A

Closes #24782 from wangyum/SPARK-27934.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-11 15:57:59 +08:00
LantaoJin 63e0711524 [SPARK-27899][SQL] Make HiveMetastoreClient.getTableObjectsByName available in ExternalCatalog/SessionCatalog API
## What changes were proposed in this pull request?

The new Spark ThriftServer SparkGetTablesOperation implemented in https://github.com/apache/spark/pull/22794 does a catalog.getTableMetadata request for every table. This can get very slow for large schemas (~50ms per table with an external Hive metastore).
Hive ThriftServer GetTablesOperation uses HiveMetastoreClient.getTableObjectsByName to get table information in bulk, but we don't expose that through our APIs that go through Hive -> HiveClientImpl (HiveClient) -> HiveExternalCatalog (ExternalCatalog) -> SessionCatalog.

If we added and exposed getTableObjectsByName through our catalog APIs, we could resolve that performance problem in SparkGetTablesOperation.

## How was this patch tested?

Add UT

Closes #24774 from LantaoJin/SPARK-27899.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-11 15:32:59 +08:00
John Zhuge dbba3a33bc [SPARK-27947][SQL] Enhance redactOptions to accept any Map type
## What changes were proposed in this pull request?

Handle the case when ParsedStatement subclass has a Map field but not of type Map[String, String].

In ParsedStatement.productIterator, `case mapArg: Map[_, _]` can match any Map type due to type erasure, thus causing `asInstanceOf[Map[String, String]]` to throw ClassCastException.

The following test reproduces the issue:
```
case class TestStatement(p: Map[String, Int]) extends ParsedStatement {
 override def output: Seq[Attribute] = Nil
 override def children: Seq[LogicalPlan] = Nil
}

TestStatement(Map("abc" -> 1)).toString
```
Changing the code to `case mapArg: Map[String, String]` will not help due to type erasure. As a matter of fact, compiler gives this warning:
```
Warning:(41, 18) non-variable type argument String in type pattern
 scala.collection.immutable.Map[String,String] (the underlying of Map[String,String])
 is unchecked since it is eliminated by erasure
case mapArg: Map[String, String] =>
```

## How was this patch tested?

Add 2 unit tests.

Closes #24800 from jzhuge/SPARK-27947.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 11:58:37 -07:00
Zhu, Lipeng 3b37bfde2a [SPARK-27949][SQL] Support SUBSTRING(str FROM n1 [FOR n2]) syntax
## What changes were proposed in this pull request?

Currently, function `substr/substring`'s usage is like `substring(string_expression, n1 [,n2])`.

But, the ANSI SQL defined the pattern for substr/substring is like `SUBSTRING(str FROM n1 [FOR n2])`. This gap makes some inconvenient when we switch to the SparkSQL.

- ANSI SQL-92: http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt

Below are the mainly DB engines to support the ANSI standard for substring.
- PostgreSQL https://www.postgresql.org/docs/9.1/functions-string.html
- MySQL https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_substring
- Redshift https://docs.aws.amazon.com/redshift/latest/dg/r_SUBSTRING.html
- Teradata https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/XnePye0Cwexw6Pny_qnxVA

**Oracle, SQL Server, Hive, Presto don't have this additional syntax.**

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24802 from lipzhu/SPARK-27949.

Authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 09:05:10 -07:00
Chaerim Yeo c1bb3316bd [SPARK-27425][SQL] Add count_if function
## What changes were proposed in this pull request?

Add `count_if` function which returns the number of records satisfying a given condition.

There is no aggregation function like this in Spark, so we need to write like
- `COUNT(CASE WHEN some_condition THEN 1 END)` or
- `SUM(CASE WHEN some_condition THEN 1 END)`, 
which looks painful.

This kind of function is already supported in Presto, BigQuery and even Excel.
- Presto: [`count_if`](https://prestodb.github.io/docs/current/functions/aggregate.html#count_if)
- BigQuery: [`countif`](https://cloud.google.com/bigquery/docs/reference/standard-sql/aggregate_functions?hl=en#countif)
- Excel: [`COUNTIF`](https://support.office.com/en-us/article/countif-function-e0de10c6-f885-4e71-abb4-1f464816df34?omkt=en-US&ui=en-US&rs=en-US&ad=US) (It is a little different from above twos)

## How was this patch tested?

This patch is tested by unit test.

Closes #24335 from cryeo/SPARK-27425.

Authored-by: Chaerim Yeo <yeochaerim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-10 19:51:55 +09:00
sandeep katta 773cfde680 [SPARK-27917][SQL] canonical form of CaseWhen object is incorrect
## What changes were proposed in this pull request?

For caseWhen Object canonicalized is not handled

for e.g let's consider below CaseWhen Object
    val attrRef = AttributeReference("ACCESS_CHECK", StringType)()
    val caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))))

caseWhenObj1.canonicalized **ouput** is as below

CASE WHEN ACCESS_CHECK#0 THEN A END (**Before Fix)**

**After Fix** : CASE WHEN none#0 THEN A END

So when there will be aliasref like below statements, semantic equals will fail. Sematic equals returns true if the canonicalized form of both the expressions are same.

val attrRef = AttributeReference("ACCESS_CHECK", StringType)()
val aliasAttrRef = attrRef.withName("access_check")
val caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))))
val caseWhenObj2 = CaseWhen(Seq((aliasAttrRef, Literal("A"))))

**assert(caseWhenObj2.semanticEquals(caseWhenObj1.semanticEquals) fails**

**caseWhenObj1.canonicalized**

Before Fix:CASE WHEN ACCESS_CHECK#0 THEN A END
After Fix: CASE WHEN none#0 THEN A END
**caseWhenObj2.canonicalized**

Before Fix:CASE WHEN access_check#0 THEN A END
After Fix: CASE WHEN none#0 THEN A END

## How was this patch tested?
Added UT

Closes #24766 from sandeep-katta/caseWhenIssue.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 00:33:47 -07:00
HyukjinKwon f984f6acfe Revert "[SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)"
## What changes were proposed in this pull request?

This PR reverts 9c4eb99c52 for the reasons below:

1. An alternative was not considered properly, https://github.com/apache/spark/pull/24734#issuecomment-500101639 https://github.com/apache/spark/pull/24734#issuecomment-500102340 https://github.com/apache/spark/pull/24734#issuecomment-499202982 - I opened a PR https://github.com/apache/spark/pull/24826

2. 9c4eb99c52 fixed timely flushing which behaviour is somewhat hacky and the timing isn't also guaranteed (in case each batch takes longer to process).

3. For pipelining for smaller batches, looks it's better to allow to configure buffer size rather than having another factor to flush

## How was this patch tested?

N/A

Closes #24827 from HyukjinKwon/revert-flush.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 08:28:31 -07:00
HyukjinKwon 6dcf09becc [SPARK-27971][SQL][R] MapPartitionsInRWithArrowExec.evaluate shouldn't eagerly read the first batch
## What changes were proposed in this pull request?

This PR is the same fix as https://github.com/apache/spark/pull/24816 but in vectorized `dapply` in SparkR.

## How was this patch tested?

Manually tested.

Closes #24818 from HyukjinKwon/SPARK-27971.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-09 11:40:20 +09:00
Gengliang Wang db0f6b4674 [SPARK-27961][SQL] DataSourceV2Relation should not have refresh method
## What changes were proposed in this pull request?

The newly added `Refresh` method in PR #24401 prevented the work of moving DataSourceV2Relation into catalyst. It calls `case table: FileTable => table.fileIndex.refresh()` while `FileTable` belongs to sql/core.

More importantly, Ryan Blue pointed out DataSourceV2Relation is immutable by design, it should not have refresh method.

## How was this patch tested?

Unit test

Closes #24815 from gengliangwang/removeRefreshTable.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 10:59:10 -07:00
Yuming Wang 2926890ffb [SPARK-27970][SQL] Support Hive 3.0 metastore
## What changes were proposed in this pull request?

It seems that some users are using Hive 3.0.0. This pr makes it support Hive 3.0 metastore.

## How was this patch tested?

unit tests

Closes #24688 from wangyum/SPARK-26145.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 15:24:07 -07:00
WeichenXu 9c4eb99c52 [SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)
## What changes were proposed in this pull request?

Flush batch timely for pandas UDF.

This could improve performance when multiple pandas UDF plans are pipelined.

When batch being flushed in time, downstream pandas UDFs will get pipelined as soon as possible, and pipeline will help hide the donwstream UDFs computation time. For example:

When the first UDF start computing on batch-3, the second pipelined UDF can start computing on batch-2, and the third pipelined UDF can start computing on batch-1.

If we do not flush each batch in time, the donwstream UDF's pipeline will lag behind too much, which may increase the total processing time.

I add flush at two places:
* JVM process feed data into python worker. In jvm side, when write one batch, flush it
* VM process read data from python worker output, In python worker side, when write one batch, flush it

If no flush, the default buffer size for them are both 65536. Especially in the ML case, in order to make realtime prediction, we will make batch size very small. The buffer size is too large for the case, which cause downstream pandas UDF pipeline lag behind too much.

### Note
* This is only applied to pandas scalar UDF.
* Do not flush for each batch. The minimum interval between two flush is 0.1 second. This avoid too frequent flushing when batch size is small. It works like:
```
        last_flush_time = time.time()
        for batch in iterator:
                writer.write_batch(batch)
                flush_time = time.time()
                if self.flush_timely and (flush_time - last_flush_time > 0.1):
                      stream.flush()
                      last_flush_time = flush_time
```

## How was this patch tested?

### Benchmark to make sure the flush do not cause performance regression
#### Test code:
```
numRows = ...
batchSize = ...

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', str(batchSize))
df = spark.range(1, numRows + 1, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 10

beg_time = time.time()
result = df.select(sum(fp1('a'))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```
#### Test Result:

 params        | Consume time (Before) | Consume time (After)
------------ | ----------------------- | ----------------------
numRows=100000000, batchSize=10000 | 23.43s | 24.64s
numRows=100000000, batchSize=1000 | 36.73s | 34.50s
numRows=10000000, batchSize=100 | 35.67s | 32.64s
numRows=1000000, batchSize=10 | 33.60s | 32.11s
numRows=100000, batchSize=1 | 33.36s | 31.82s

### Benchmark pipelined pandas UDF
#### Test code:
```
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))

```
#### Test Result:

**Before**: consume time: 63.57s
**After**: consume time: 32.43s
**So the PR improve performance by make downstream UDF get pipelined early.**

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

Closes #24734 from WeichenXu123/improve_pandas_udf_pipeline.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 14:02:43 -07:00
Liang-Chi Hsieh 527d936049 [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation
## What changes were proposed in this pull request?

When using `from_avro` to deserialize avro data to catalyst StructType format, if `ConvertToLocalRelation` is applied at the time, `from_avro` produces only the last value (overriding previous values).

The cause is `AvroDeserializer` reuses output row for StructType. Normally, it should be fine in Spark SQL. But `ConvertToLocalRelation` just uses `InterpretedProjection` to project local rows. `InterpretedProjection` creates new row for each output thro, it includes the same nested row object from `AvroDeserializer`. By the end, converted local relation has only last value.

I think there're two possible options:

1. Make `AvroDeserializer` output new row for StructType.
2. Use `InterpretedMutableProjection` in `ConvertToLocalRelation` and call `copy()` on output rows.

Option 2 is chose because previously `ConvertToLocalRelation` also creates new rows, this `InterpretedMutableProjection` + `copy()` shoudn't bring too much performance penalty. `ConvertToLocalRelation` should be arguably less critical, compared with `AvroDeserializer`.

## How was this patch tested?

Added test.

Closes #24805 from viirya/SPARK-27798.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-07 13:47:36 -07:00
Ryan Blue b30655bdef [SPARK-27965][SQL] Add extractors for v2 catalog transforms.
## What changes were proposed in this pull request?

Add extractors for v2 catalog transforms.

These extractors are used to match transforms that are equivalent to Spark's internal case classes. This makes it easier to work with v2 transforms.

## How was this patch tested?

Added test suite for the new extractors.

Closes #24812 from rdblue/SPARK-27965-add-transform-extractors.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 00:20:36 -07:00
liwensun eee3467b1e [SPARK-27938][SQL] Remove feature flag LEGACY_PASS_PARTITION_BY_AS_OPTIONS
## What changes were proposed in this pull request?
In PR https://github.com/apache/spark/pull/24365, we pass in the partitionBy columns as options in `DataFrameWriter`.  To make this change less intrusive for a patch release, we added a feature flag `LEGACY_PASS_PARTITION_BY_AS_OPTIONS` with the default to be false.

For 3.0, we should just do the correct behavior for DSV1, i.e., always passing partitionBy as options, and remove this legacy feature flag.

## How was this patch tested?
Existing tests.

Closes #24784 from liwensun/SPARK-27453-default.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-07 11:33:58 +09:00
Xiangrui Meng 4d770db0eb [SPARK-27968] ArrowEvalPythonExec.evaluate shouldn't eagerly read the first row
## What changes were proposed in this pull request?

Issued fixed in https://github.com/apache/spark/pull/24734 but that PR might takes longer to merge.

## How was this patch tested?

It should pass existing unit tests.

Closes #24816 from mengxr/SPARK-27968.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-06 15:45:44 -07:00
Yuming Wang eadb53824d [SPARK-27918][SQL] Port boolean.sql
## What changes were proposed in this pull request?

This PR is to port boolean.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/boolean.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/boolean.out

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
- [SPARK-27931](https://issues.apache.org/jira/browse/SPARK-27931): Accept 'on' and 'off' as input for boolean data type / Trim the string when cast to boolean type / Accept unique prefixes thereof
- [SPARK-27924](https://issues.apache.org/jira/browse/SPARK-27924): Support E061-14: Search Conditions

Also, found an inconsistent behavior:
- [SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Unsupported input throws an exception in PostgreSQL but Spark accepts it and sets the value to `NULL`, for example:
```sql
SELECT bool 'test' AS error; -- SELECT boolean('test') AS error;
```

## How was this patch tested?

N/A

Closes #24767 from wangyum/SPARK-27918.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-06 10:57:10 -07:00
Yuming Wang 4de96493ae [SPARK-27883][SQL] Port AGGREGATES.sql [Part 2]
## 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_BETA1/src/test/regress/sql/aggregates.sql#L145-L350

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

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

- [SPARK-27877](https://issues.apache.org/jira/browse/SPARK-27877): Implement SQL-standard LATERAL subqueries
- [SPARK-27878](https://issues.apache.org/jira/browse/SPARK-27878): Support ARRAY(sub-SELECT) expressions
- [SPARK-27879](https://issues.apache.org/jira/browse/SPARK-27879): Implement bitwise integer aggregates(BIT_AND and BIT_OR)
- [SPARK-27880](https://issues.apache.org/jira/browse/SPARK-27880): Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY)

## How was this patch tested?

N/A

Closes #24743 from wangyum/SPARK-27883.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-06 09:28:59 -07:00
Ryan Blue d1371a2dad [SPARK-27964][SQL] Move v2 catalog update methods to CatalogV2Util
## What changes were proposed in this pull request?

Move methods that implement v2 catalog operations to CatalogV2Util so they can be used in #24768.

## How was this patch tested?

Behavior is validated by existing tests.

Closes #24813 from rdblue/SPARK-27964-add-catalog-v2-util.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-05 19:44:53 -07:00
Jordan Sanders 20e8843350 [MINOR][SQL] Skip warning if JOB_SUMMARY_LEVEL is set to NONE
## What changes were proposed in this pull request?

I believe the log message: `Committer $committerClass is not a ParquetOutputCommitter and cannot create job summaries. Set Parquet option ${ParquetOutputFormat.JOB_SUMMARY_LEVEL} to NONE.` is at odds with the `if` statement that logs the warning. Despite the instructions in the warning, users still encounter the warning if `JOB_SUMMARY_LEVEL` is already set to `NONE`.

This pull request introduces a change to skip logging the warning if `JOB_SUMMARY_LEVEL` is set to `NONE`.

## How was this patch tested?

I built to make sure everything still compiled and I ran the existing test suite. I didn't feel it was worth the overhead to add a test to make sure a log message does not get logged, but if reviewers feel differently, I can add one.

Closes #24808 from jmsanders/master.

Authored-by: Jordan Sanders <jmsanders@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-05 14:57:36 -07:00
Ryan Blue 5d6758c0e7 [SPARK-27857][SQL] Move ALTER TABLE parsing into Catalyst
## What changes were proposed in this pull request?

This moves parsing logic for `ALTER TABLE` into Catalyst and adds parsed logical plans for alter table changes that use multi-part identifiers. This PR is similar to SPARK-27108, PR #24029, that created parsed logical plans for create and CTAS.

* Create parsed logical plans
* Move parsing logic into Catalyst's AstBuilder
* Convert to DataSource plans in DataSourceResolution
* Parse `ALTER TABLE ... SET LOCATION ...` separately from the partition variant
* Parse `ALTER TABLE ... ALTER COLUMN ... [TYPE dataType] [COMMENT comment]` [as discussed on the dev list](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Syntax-for-table-DDL-td25197.html#a25270)
* Parse `ALTER TABLE ... RENAME COLUMN ... TO ...`
* Parse `ALTER TABLE ... DROP COLUMNS ...`

## How was this patch tested?

* Added new tests in Catalyst's `DDLParserSuite`
* Moved converted plan tests from SQL `DDLParserSuite` to `PlanResolutionSuite`
* Existing tests for regressions

Closes #24723 from rdblue/SPARK-27857-add-alter-table-statements-in-catalyst.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 13:21:30 -07:00
Jacek Laskowski 6c28ef144d [SPARK-27933][SS] Extracting common purge behaviour to the parent StreamExecution
Extracting the common purge "behaviour" to the parent StreamExecution.

## How was this patch tested?

No added behaviour so relying on existing tests.

Closes #24781 from jaceklaskowski/StreamExecution-purge.

Authored-by: Jacek Laskowski <jacek@japila.pl>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-05 12:39:31 -05:00
Wenchen Fan 8b6232b119 [SPARK-27521][SQL] Move data source v2 to catalyst module
## What changes were proposed in this pull request?

Currently we are in a strange status that, some data source v2 interfaces(catalog related) are in sql/catalyst, some data source v2 interfaces(Table, ScanBuilder, DataReader, etc.) are in sql/core.

I don't see a reason to keep data source v2 API in 2 modules. If we should pick one module, I think sql/catalyst is the one to go.

Catalyst module already has some user-facing stuff like DataType, Row, etc. And we have to update `Analyzer` and `SessionCatalog` to support the new catalog plugin, which needs to be in the catalyst module.

This PR can solve the problem we have in https://github.com/apache/spark/pull/24246

## How was this patch tested?

existing tests

Closes #24416 from cloud-fan/move.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 09:55:55 -07:00
Yuming Wang 3f102a8229 [SPARK-27749][SQL] hadoop-3.2 support hive-thriftserver
## What changes were proposed in this pull request?

This PR mainly makes the following changes to make `hadoop-3.2` support `sql/hive-thriftserver`:
1. Upgrade [`TCLIService.thrift`](https://github.com/apache/hive/blob/rel/release-2.3.5/service-rpc/if/TCLIService.thrift) and related code to Hive 2.3.5 because of [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442)(Note that we only migrate code without adding features, such as [HIVE-4924](https://issues.apache.org/jira/browse/HIVE-4924) and [HIVE-15473](https://issues.apache.org/jira/browse/HIVE-15473)).
2. Use slf4j as logging facade because of [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237).
3. Port [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) to compatible with Hive 2.3.

## How was this patch tested?

Exiting test

Closes #24628 from wangyum/SPARK-27749.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:40:05 -07:00
LantaoJin 18834e85d0 [SPARK-27899][SQL] Refactor getTableOption() to extract a common method
## What changes were proposed in this pull request?

This is a part of #24774, to reduce the code changes made by that.

## How was this patch tested?

Exist UTs.

Closes #24803 from LantaoJin/SPARK-27899_refactor.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:36:25 -07:00
Ryan Blue de73a54269 [SPARK-27909][SQL] Do not run analysis inside CTE substitution
## What changes were proposed in this pull request?

This updates CTE substitution to avoid needing to run all resolution rules on each substituted expression. Running resolution rules was previously used to avoid infinite recursion. In the updated rule, CTE plans are substituted as sub-queries from right to left. Using this scope-based order, it is not necessary to replace multiple CTEs at the same time using `resolveOperatorsDown`. Instead, `resolveOperatorsUp` is used to replace each CTE individually.

By resolving using `resolveOperatorsUp`, this no longer needs to run all analyzer rules on each substituted expression. Previously, this was done to apply `ResolveRelations`, which would throw an `AnalysisException` for all unresolved relations so that unresolved relations that may cause recursive substitutions were not left in the plan. Because this is no longer needed, `ResolveRelations` no longer needs to throw `AnalysisException` and resolution can be done in multiple rules.

## How was this patch tested?

Existing tests in `SQLQueryTestSuite`, `cte.sql`.

Closes #24763 from rdblue/SPARK-27909-fix-cte-substitution.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-04 14:46:13 -07:00
David Vogelbacher f9ca8ab196 [SPARK-27805][PYTHON] Propagate SparkExceptions during toPandas with arrow enabled
## What changes were proposed in this pull request?
Similar to https://github.com/apache/spark/pull/24070, we now propagate SparkExceptions that are encountered during the collect in the java process to the python process.

Fixes https://jira.apache.org/jira/browse/SPARK-27805

## How was this patch tested?
Added a new unit test

Closes #24677 from dvogelbacher/dv/betterErrorMsgWhenUsingArrow.

Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-04 10:10:27 -07:00
williamwong d5715a9b23 [SPARK-27772][SQL][TEST] Refactor SQLTestUtils to use tryWithSafeFinally
## What changes were proposed in this pull request?

The current `SQLTestUtils` created many `withXXX` utility functions to clean up tables/views/caches created for testing purpose. Java's `try-with-resources` statement does something similar, but it does not mask exception throwing in the try block with any exception caught in the 'close()' statement. Exception caught in the 'close()' statement would add as a suppressed exception instead.

This PR standardizes those 'withXXX' function to use`Utils.tryWithSafeFinally` function, which does something similar to Java's try-with-resources statement. The purpose of this proposal is to help developers to identify what actually breaks their tests.

## How was this patch tested?
Existing testcases.

Closes #24747 from William1104/feature/SPARK-27772-2.

Lead-authored-by: williamwong <william1104@gmail.com>
Co-authored-by: William Wong <william1104@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-04 09:26:24 -05:00
Michael Chirico 3ddc26ddd8 [MINOR][DOCS] Add a clarifying note to str_to_map documentation
I was quite surprised by the following behavior:

`SELECT str_to_map('1:2|3:4', '|')`

vs

`SELECT str_to_map(replace('1:2|3:4', '|', ','))`

The documentation does not make clear at all what's going on here, but a [dive into the source code shows](fa0d4bf699/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala (L461-L466)) that `split` is being used and in turn the interpretation of `split`'s arguments as RegEx is clearly documented.

## What changes were proposed in this pull request?

Documentation clarification

## How was this patch tested?

N/A

Closes #23888 from MichaelChirico/patch-2.

Authored-by: Michael Chirico <michaelchirico4@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-04 16:58:25 +09:00
Gengliang Wang d1937c1479 [SPARK-27926][SQL] Allow altering table add columns with CSVFileFormat/JsonFileFormat provider
## What changes were proposed in this pull request?

In the previous work of csv/json migration, CSVFileFormat/JsonFileFormat is removed in the table provider whitelist of `AlterTableAddColumnsCommand.verifyAlterTableAddColumn`:
https://github.com/apache/spark/pull/24005
https://github.com/apache/spark/pull/24058

This is regression. If a table is created with Provider `org.apache.spark.sql.execution.datasources.csv.CSVFileFormat` or `org.apache.spark.sql.execution.datasources.json.JsonFileFormat`, Spark should allow the "alter table add column" operation.

## How was this patch tested?

Unit test

Closes #24776 from gengliangwang/v1Table.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-03 23:51:05 -07:00
Dongjoon Hyun 8486680b34 [SPARK-24544][SQL][FOLLOWUP] Remove a wrong warning on Hive fallback lookup
## What changes were proposed in this pull request?

This PR is a follow-up of https://github.com/apache/spark/pull/21790 which causes a regression to show misleading warnings always at first invocation for all Hive function. Hive fallback lookup should not be warned. It's a normal process in function lookups.

**CURRENT (Showing `NoSuchFunctionException` and working)**
```scala
scala> sql("select histogram_numeric(a,2) from values(1) T(a)").show
19/06/02 22:02:10 WARN HiveSessionCatalog: Encountered a failure during looking up
function: org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException:
Undefined function: 'histogram_numeric'. This function is neither a registered temporary
function nor a permanent function registered in the database 'default'.;
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.failFunctionLookup(SessionCatalog.scala:1234)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1302)
...
+------------------------+
|histogram_numeric( a, 2)|
+------------------------+
|            [[1.0, 1.0]]|
+------------------------+
```

## How was this patch tested?

Manually execute the above query.

Closes #24773 from dongjoon-hyun/SPARK-24544.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-03 00:04:00 -07:00
HyukjinKwon 8b18ef5c7b [MINOR] Avoid hardcoded py4j-0.10.8.1-src.zip in Scala
## What changes were proposed in this pull request?

This PR targets to deduplicate hardcoded `py4j-0.10.8.1-src.zip` in order to make py4j upgrade easier.

## How was this patch tested?

N/A

Closes #24770 from HyukjinKwon/minor-py4j-dedup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:23:17 -07:00
Dongjoon Hyun 809821a283 [SPARK-27920][SQL][TEST] Add interceptParseException test utility function
## What changes were proposed in this pull request?

This PR aims to add `interceptParseException` test utility function to `AnalysisTest` to reduce the duplications of `intercept` functions.

## How was this patch tested?

Pass the Jenkins with the updated test suites.

Closes #24769 from dongjoon-hyun/SPARK-27920.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:11:35 -07:00
Yuming Wang d53b61c311 [SPARK-27831][SQL][TEST] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Please note that this pr need test with `maven` and `sbt`.

Closes #24751 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 20:23:08 -07:00
Liang-Chi Hsieh 2a88fffacb [SPARK-27873][SQL] columnNameOfCorruptRecord should not be checked with column names in CSV header when disabling enforceSchema
## What changes were proposed in this pull request?

If we want to keep corrupt record when reading CSV, we provide a new column into the schema, that is `columnNameOfCorruptRecord`. But this new column isn't actually a column in CSV header. So if `enforceSchema` is disabled, `CSVHeaderChecker` throws a exception complaining that number of column in CSV header isn't equal to that in the schema.

## How was this patch tested?

Added test.

Closes #24757 from viirya/SPARK-27873.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 11:09:26 +09:00
HyukjinKwon f5317f10b2 [SPARK-27893][SQL][PYTHON] Create an integrated test base for Python, Scalar Pandas, Scala UDF by sql files
## What changes were proposed in this pull request?

This PR targets to add an integrated test base for various UDF test cases so that Scalar UDF, Python UDF and Scalar Pandas UDFs can be tested in SBT & Maven tests.

### Problem

One of the problems we face is that: `ExtractPythonUDFs` (for Python UDF and Scalar Pandas UDF) has unevaluable expressions that always has to be wrapped with special plans. This special rule seems producing many issues, for instance, SPARK-27803, SPARK-26147, SPARK-26864, SPARK-26293, SPARK-25314 and SPARK-24721.

### Why do we have less test cases dedicated for SQL and plans with Python UDFs?

We have virtually no such SQL (or plan) dedicated tests in PySpark to catch such issues because:
  - A developer should know all the analyzer, the optimizer, SQL, PySpark, Py4J and version differences in Python to write such good test cases
  - To test plans, we should access to plans in JVM via Py4J which is tricky, messy and duplicates Scala test cases
  - Usually we just add end-to-end test cases in PySpark therefore there are not so many dedicated examples to refer to write in PySpark

It is also a non-trivial overhead to switch test base and method (IMHO).

### How does this PR fix?

This PR adds Python UDF and Scalar Pandas UDF into our `*.sql` file based test base in runtime of SBT / Maven test cases. It generates Python-pickled instance (consisting of return type and Python native function) that is used in Python or Scalar Pandas UDF and directly brings into JVM.

After that, (we don't interact via Py4J) run the tests directly in JVM - we can just register and run Python UDF and Scalar Pandas UDF in JVM.

Currently, I only integrated this change into SQL file based testing. This is how works with test files under `udf` directory:

After the test files under 'inputs/udf' directory are detected, it creates three test cases:
  - Scala UDF test case with a Scalar UDF registered named 'udf'.
  - Python UDF test case with a Python UDF registered named 'udf' iff Python executable and pyspark are available.
  - Scalar Pandas UDF test case with a Scalar Pandas UDF registered named 'udf' iff Python executable, pandas, pyspark and pyarrow are available.

Therefore, UDF test cases should have single input and output files but executed by three different types of UDFs.

For instance,

```sql
CREATE TEMPORARY VIEW ta AS
SELECT udf(a) AS a, udf('a') AS tag FROM t1
UNION ALL
SELECT udf(a) AS a, udf('b') AS tag FROM t2;

CREATE TEMPORARY VIEW tb AS
SELECT udf(a) AS a, udf('a') AS tag FROM t3
UNION ALL
SELECT udf(a) AS a, udf('b') AS tag FROM t4;

SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag;
```

will be ran 3 times with Scalar UDF, Python UDF and Scalar Pandas UDF each.

### Appendix

Plus, this PR adds `IntegratedUDFTestUtils` which enables to test and execute Python UDF and Scalar Pandas UDFs as below:

To register Python UDF in SQL:

```scala
IntegratedUDFTestUtils.registerTestUDF(TestPythonUDF(name = "udf"), spark)
```

To register Scalar Pandas UDF in SQL:

```scala
IntegratedUDFTestUtils.registerTestUDF(TestScalarPandasUDF(name = "udf"), spark)
```

 To use it in Scala API:

```scala
spark.select(expr("udf(1)").show()
```

 To use it in SQL:

```scala
sql("SELECT udf(1)").show()
```

This util could be used in the future for better coverage with Scala API combinations as well.

## How was this patch tested?

Tested via the command below:

```bash
build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/udf-inner-join.sql"
```

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (5 seconds, 47 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF (4 seconds, 335 milliseconds)
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF (5 seconds, 423 milliseconds)
```

[python] unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 577 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF is skipped because [pyton] and/or pyspark were not available. !!! IGNORED !!!
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [pyton]. !!! IGNORED !!!
```

pyspark unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 991 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF is skipped because [python] and/or pyspark were not available. !!! IGNORED !!!
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python]. !!! IGNORED !!!
```

pandas and/or pyarrow unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 713 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF (3 seconds, 89 milliseconds)
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pandas and/or pyarrow were not available in [python]. !!! IGNORED !!!
```

Closes #24752 from HyukjinKwon/udf-tests.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 10:03:36 +09:00
HyukjinKwon db48da87f0 [SPARK-27834][SQL][R][PYTHON] Make separate PySpark/SparkR vectorization configurations
## What changes were proposed in this pull request?

`spark.sql.execution.arrow.enabled` was added when we add PySpark arrow optimization.
Later, in the current master, SparkR arrow optimization was added and it's controlled by the same configuration `spark.sql.execution.arrow.enabled`.

There look two issues about this:

1. `spark.sql.execution.arrow.enabled` in PySpark was added from 2.3.0 whereas SparkR optimization was added 3.0.0. The stability is different so it's problematic when we change the default value for one of both optimization first.

2. Suppose users want to share some JVM by PySpark and SparkR. They are currently forced to use the optimization for all or none if the configuration is set globally.

This PR proposes two separate configuration groups for PySpark and SparkR about Arrow optimization:

- Deprecate `spark.sql.execution.arrow.enabled`
- Add `spark.sql.execution.arrow.pyspark.enabled` (fallback to `spark.sql.execution.arrow.enabled`)
- Add `spark.sql.execution.arrow.sparkr.enabled`
- Deprecate `spark.sql.execution.arrow.fallback.enabled`
- Add `spark.sql.execution.arrow.pyspark.fallback.enabled ` (fallback to `spark.sql.execution.arrow.fallback.enabled`)

Note that `spark.sql.execution.arrow.maxRecordsPerBatch` is used within JVM side for both.
Note that `spark.sql.execution.arrow.fallback.enabled` was added due to behaviour change. We don't need it in SparkR - SparkR side has the automatic fallback.

## How was this patch tested?

Manually tested and some unittests were added.

Closes #24700 from HyukjinKwon/separate-sparkr-arrow.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 10:01:37 +09:00
Ajith 3806887afb [SPARK-27907][SQL] HiveUDAF should return NULL in case of 0 rows
## What changes were proposed in this pull request?

When query returns zero rows, the HiveUDAFFunction throws NPE

## CASE 1:
create table abc(a int)
select histogram_numeric(a,2) from abc // NPE
```
Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 0, localhost, executor driver): java.lang.NullPointerException
	at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:471)
	at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:315)
	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.eval(interfaces.scala:543)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:231)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:122)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```

## CASE 2:
create table abc(a int)
insert into abc values (1)
select histogram_numeric(a,2) from abc where a=3 // NPE

```
Job aborted due to stage failure: Task 0 in stage 4.0 failed 1 times, most recent failure: Lost task 0.0 in stage 4.0 (TID 5, localhost, executor driver): java.lang.NullPointerException
at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:477)
at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:315)
at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.serializeAggregateBufferInPlace(interfaces.scala:570)
at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$6(AggregationIterator.scala:254)
at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:94)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:122)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```

Hence add a check not avoid NPE

## How was this patch tested?

Added new UT case

Closes #24762 from ajithme/hiveudaf.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 10:54:21 -07:00
Marco Gaido 93db7b870d [SPARK-27684][SQL] Avoid conversion overhead for primitive types
## What changes were proposed in this pull request?

As outlined in the JIRA by JoshRosen, our conversion mechanism from catalyst types to scala ones is pretty inefficient for primitive data types. Indeed, in these cases, most of the times we are adding useless calls to `identity` function or anyway to functions which return the same value. Using the information we have when we generate the code, we can avoid most of these overheads.

## How was this patch tested?

Here is a simple test which shows the benefit that this PR can bring:
```
test("SPARK-27684: perf evaluation") {
    val intLongUdf = ScalaUDF(
      (a: Int, b: Long) => a + b, LongType,
      Literal(1) :: Literal(1L) :: Nil,
      true :: true :: Nil,
      nullable = false)

    val plan = generateProject(
      MutableProjection.create(Alias(intLongUdf, s"udf")() :: Nil),
      intLongUdf)
    plan.initialize(0)

    var i = 0
    val N = 100000000
    val t0 = System.nanoTime()
    while(i < N) {
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      plan(EmptyRow).get(0, intLongUdf.dataType)
      i += 1
    }
    val t1 = System.nanoTime()
    println(s"Avg time: ${(t1 - t0).toDouble / N} ns")
  }
```
The output before the patch is:
```
Avg time: 51.27083294 ns
```
after, we get:
```
Avg time: 11.85874227 ns
```
which is ~5X faster.

Moreover a benchmark has been added for Scala UDF. The output after the patch can be seen in this PR, before the patch, the output was:
```
================================================================================================
UDF with mixed input types
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int/string to string:       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int/string to string wholestage off            257            287          42          0,4        2569,5       1,0X
long/nullable int/string to string wholestage on            158            172          18          0,6        1579,0       1,6X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int/string to option:       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int/string to option wholestage off            104            107           5          1,0        1037,9       1,0X
long/nullable int/string to option wholestage on             80             92          12          1,2         804,0       1,3X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int to primitive:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int to primitive wholestage off             71             76           7          1,4         712,1       1,0X
long/nullable int to primitive wholestage on             64             71           6          1,6         636,2       1,1X

================================================================================================
UDF with primitive types
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int to string:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int to string wholestage off             60             60           0          1,7         600,3       1,0X
long/nullable int to string wholestage on             55             64           8          1,8         551,2       1,1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int to option:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int to option wholestage off             66             73           9          1,5         663,0       1,0X
long/nullable int to option wholestage on             30             32           2          3,3         300,7       2,2X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
long/nullable int/string to primitive:    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
long/nullable int/string to primitive wholestage off             32             35           5          3,2         316,7       1,0X
long/nullable int/string to primitive wholestage on             41             68          17          2,4         414,0       0,8X
```
The improvements are particularly visible in the second case, ie. when only primitive types are used as inputs.

Closes #24636 from mgaido91/SPARK-27684.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-05-30 17:09:19 -07:00
Gengliang Wang 49e7387741 [SPARK-27849][SQL][FOLLOWUP][TEST-MAVEN] Fix the testing regex in DataSourceScanRedactionTest
## What changes were proposed in this pull request?

As explained in https://github.com/apache/spark/pull/24719#pullrequestreview-243064785, the regex `file:/[\\w-_/]+` contains possible characters I have met in the Jenkins tests.
However, we still miss the  `.` symbol:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/6415/testReport/junit/org.apache.spark.sql.execution/DataSourceV2ScanExecRedactionSuite/treeString_is_redacted/ :
```
orc *********(redacted).7/sql/core/target/tmp/spark-7ff5f81d-069a-4b5d-9d9a-808addeef115
```

This PR is to fix it by matching any character except `]` or spaces.
## How was this patch tested?

Unit test

Closes #24745 from gengliangwang/fixRegex.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-30 13:45:12 -07:00
Dongjoon Hyun 955eef95b3 Revert "[SPARK-27831][SQL][TEST][test-hadoop3.2] Move Hive test jars to maven dependency"
This reverts commit 24180c00e0.
2019-05-30 10:06:55 -07:00
John Zhuge a44b00dfe0 [SPARK-27813][SQL] DataSourceV2: Add DropTable logical operation
## What changes were proposed in this pull request?

Support DROP TABLE from V2 catalogs.
Move DROP TABLE into catalyst.
Move parsing tests for DROP TABLE/VIEW to PlanResolutionSuite to validate existing behavior.
Add new tests fo catalyst parser suite.
Separate DROP VIEW into different code path from DROP TABLE.
Move DROP VIEW into catalyst as a new operator.
Add a meaningful exception to indicate view is not currently supported in v2 catalog.

## How was this patch tested?

New unit tests.
Existing unit tests in catalyst and sql core.

Closes #24686 from jzhuge/SPARK-27813-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-31 00:56:07 +08:00
Yuming Wang db3e746b64 [SPARK-27875][CORE][SQL][ML][K8S] Wrap all PrintWriter with Utils.tryWithResource
## What changes were proposed in this pull request?

This pr wrap all `PrintWriter` with `Utils.tryWithResource` to prevent resource leak.

## How was this patch tested?

Existing test

Closes #24739 from wangyum/SPARK-27875.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-30 19:54:32 +09:00
John Zhuge 953b8e8206 [SPARK-26946][SQL][FOLLOWUP] Require lookup function
## What changes were proposed in this pull request?

Require the lookup function with interface LookupCatalog. Rationale is in the review comments below.

Make `Analyzer` abstract. BaseSessionStateBuilder and HiveSessionStateBuilder implements lookupCatalog with a call to SparkSession.catalog().

Existing test cases and those that don't need catalog lookup will use a newly added `TestAnalyzer` with a default lookup function that throws` CatalogNotFoundException("No catalog lookup function")`.

Rewrote the unit test for LookupCatalog to demonstrate the interface can be used anywhere, not just Analyzer.

Removed Analyzer parameter `lookupCatalog` because we can override in the following manner:
```
new Analyzer() {
  override def lookupCatalog(name: String): CatalogPlugin = ???
}
```

## How was this patch tested?

Existing unit tests.

Closes #24689 from jzhuge/SPARK-26946-follow.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-30 09:22:42 +08:00
Gengliang Wang c1007c2f7c [SPARK-27849][SQL] Redact treeString of FileTable and DataSourceV2ScanExecBase
## What changes were proposed in this pull request?

To follow https://github.com/apache/spark/pull/17397, the output of FileTable and DataSourceV2ScanExecBase can contain sensitive information (like Amazon keys). Such information should not end up in logs, or be exposed to non-privileged users.

This PR is to add a redaction facility for these outputs to resolve the issue. A user can enable this by setting a regex in the same spark.redaction.string.regex configuration as V1.
## How was this patch tested?

Unit test

Closes #24719 from gengliangwang/RedactionSuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-29 13:32:21 -07:00
Josh Rosen 19aaf0f784 [SPARK-27829][SQL] In Dataset.joinWith() inner joins, don't nest data before shuffling
## What changes were proposed in this pull request?

In order to support outer joins with null top-level objects, SPARK-15441 modified Dataset.joinWith to project both inputs into single-column structs prior to the join.

For inner joins, however, this step is unnecessary and actually harms performance: performing the nesting before the join increases the shuffled data size. As an optimization for inner joins only, we can move this nesting to occur after the join (effectively switching back to the pre-SPARK-15441 behavior; see #13425).

## How was this patch tested?

Existing tests, which I strengthened to also make assertions about the join result's nullability (since this guards against a bug I almost introduced during prototyping).

Here's a quick `spark-shell` experiment demonstrating the reduction in shuffle size:

```scala
// With --conf spark.shuffle.compress=false
sql("set spark.sql.autoBroadcastJoinThreshold=-1") // for easier shuffle measurements
case class Foo(a: Long, b: Long)
val left = spark.range(10000).map(x => Foo(x, x))
val right = spark.range(10000).map(x => Foo(x, x))
left.joinWith(right, left("a") === right("a"), "inner").rdd.count()
left.joinWith(right, left("a") === right("a"), "left").rdd.count()
```

With inner join (which benefits from this PR's optimization) we shuffle 546.9 KiB. With left outer join (whose plan hasn't changed, therefore being a representation of the state before this PR) we shuffle 859.4 KiB. Shuffle compression (which is enabled by default) narrows this gap a bit: with compression, outer joins shuffle about 12% more than inner joins.

Closes #24693 from JoshRosen/fast-join-with-for-inner-joins.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-29 16:12:24 +08:00
Yuming Wang 67582fdbfe [SPARK-27737][SQL][FOLLOW-UP] Move sql/hive-thriftserver/v2.3.4 to sql/hive-thriftserver/v2.3.5
## What changes were proposed in this pull request?

This pr moves `sql/hive-thriftserver/v2.3.4` to `sql/hive-thriftserver/v2.3.5` based on ([comment](https://github.com/apache/spark/pull/24628#issuecomment-496459258)).

## How was this patch tested?

N/A

Closes #24728 from wangyum/SPARK-27737-thriftserver.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-28 16:08:58 -07:00
Yuming Wang 4e61de4380 [SPARK-27863][SQL] Metadata files and temporary files should not be counted as data files
## What changes were proposed in this pull request?
[`DataSourceUtils.isDataPath(path)`](https://github.com/apache/spark/blob/v2.4.3/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala#L95) should be `DataSourceUtils.isDataPath(status.getPath)`.

This pr fix this issue.

## How was this patch tested?

unit tests

Closes #24725 from wangyum/SPARK-27863.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-28 09:28:35 -07:00
gengjiaan c30b5297bc [SPARK-27776][SQL] Avoid duplicate Java reflection in DataSource.
## What changes were proposed in this pull request?

I checked the code of
`org.apache.spark.sql.execution.datasources.DataSource`
, there exists duplicate Java reflection.
`sourceSchema`,`createSource`,`createSink`,`resolveRelation`,`writeAndRead`, all the methods call the `providingClass.getConstructor().newInstance()`.
The instance of `providingClass` is stateless, such as:
`KafkaSourceProvider`
`RateSourceProvider`
`TextSocketSourceProvider`
`JdbcRelationProvider`
`ConsoleSinkProvider`

AFAIK, Java reflection will result in significant performance issue.
The oracle website [https://docs.oracle.com/javase/tutorial/reflect/index.html](https://docs.oracle.com/javase/tutorial/reflect/index.html) contains some performance description about Java reflection:

```
Performance Overhead
Because reflection involves types that are dynamically resolved, certain Java virtual machine optimizations can not be performed. Consequently, reflective operations have slower performance than their non-reflective counterparts, and should be avoided in sections of code which are called frequently in performance-sensitive applications.
```

I have found some performance cost test of Java reflection as follows:
[https://blog.frankel.ch/performance-cost-of-reflection/](https://blog.frankel.ch/performance-cost-of-reflection/) contains performance cost test.
[https://stackoverflow.com/questions/435553/java-reflection-performance](https://stackoverflow.com/questions/435553/java-reflection-performance) has a discussion of java reflection.

So I think should avoid duplicate Java reflection and reuse the instance of `providingClass`.

## How was this patch tested?

Exists UT.

Closes #24647 from beliefer/optimize-DataSource.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-28 09:26:06 -05:00
wenxuanguan 35952cb42c [SPARK-27859][SS] Use efficient sorting instead of .sorted.reverse sequence
## What changes were proposed in this pull request?

descending sort in HDFSMetadataLog.getLatest instead of two action of ascending sort and reverse

## How was this patch tested?

Jenkins

Closes #24711 from wenxuanguan/bug-fix-hdfsmetadatalog.

Authored-by: wenxuanguan <choose_home@126.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-27 21:53:23 -07:00
Wenchen Fan 6506616b97 [SPARK-27803][SQL][PYTHON] Fix column pruning for Python UDF
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22104 , we create the python-eval nodes at the end of the optimization phase, which causes a problem.

After the main optimization batch, Filter and Project nodes are usually pushed to the bottom, near the scan node. However, if we extract Python UDFs from Filter/Project, and create a python-eval node under Filter/Project, it will break column pruning/filter pushdown of the scan node.

There are some hacks in the `ExtractPythonUDFs` rule, to duplicate the column pruning and filter pushdown logic. However, it has some bugs as demonstrated in the new test case(only column pruning is broken). This PR removes the hacks and re-apply the column pruning and filter pushdown rules explicitly.

**Before:**

```
...
== Analyzed Logical Plan ==
a: bigint
Project [a#168L]
+- Filter dummyUDF(a#168L)
   +- Relation[a#168L,b#169L] parquet

== Optimized Logical Plan ==
Project [a#168L]
+- Project [a#168L, b#169L]
   +- Filter pythonUDF0#174: boolean
      +- BatchEvalPython [dummyUDF(a#168L)], [a#168L, b#169L, pythonUDF0#174]
         +- Relation[a#168L,b#169L] parquet

== Physical Plan ==
*(2) Project [a#168L]
+- *(2) Project [a#168L, b#169L]
   +- *(2) Filter pythonUDF0#174: boolean
      +- BatchEvalPython [dummyUDF(a#168L)], [a#168L, b#169L, pythonUDF0#174]
         +- *(1) FileScan parquet [a#168L,b#169L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/_1/bzcp960d0hlb988k90654z2w0000gp/T/spark-798bae3c-a2..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint,b:bigint>
```

**After:**

```
...
== Analyzed Logical Plan ==
a: bigint
Project [a#168L]
+- Filter dummyUDF(a#168L)
   +- Relation[a#168L,b#169L] parquet

== Optimized Logical Plan ==
Project [a#168L]
+- Filter pythonUDF0#174: boolean
   +- BatchEvalPython [dummyUDF(a#168L)], [pythonUDF0#174]
      +- Project [a#168L]
         +- Relation[a#168L,b#169L] parquet

== Physical Plan ==
*(2) Project [a#168L]
+- *(2) Filter pythonUDF0#174: boolean
   +- BatchEvalPython [dummyUDF(a#168L)], [pythonUDF0#174]
      +- *(1) FileScan parquet [a#168L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/_1/bzcp960d0hlb988k90654z2w0000gp/T/spark-9500cafb-78..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint>
```

## How was this patch tested?

new test

Closes #24675 from cloud-fan/python.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-27 21:39:59 +09:00
Dilip Biswal 5060647bb1 [SPARK-27782][SQL] Use '#' to mark expression id embedded in the name field of SubqueryExec operator
## What changes were proposed in this pull request?
This is a minor pr to use `#` as a marker for expression id that is embedded in the name field of SubqueryExec operator.

## How was this patch tested?
Added a small test in SubquerySuite.

Closes #24652 from dilipbiswal/subquery-name.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 20:47:25 -07:00
Yuming Wang 447bfdec83 [SPARK-27844][SQL] Avoid hard-coded config: spark.rdd.parallelListingThreshold in SQL module
## What changes were proposed in this pull request?

Avoid hard-coded config: `spark.rdd.parallelListingThreshold`.

## How was this patch tested?

N/A

Closes #24708 from wangyum/spark.rdd.parallelListingThreshold.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 09:00:01 -07:00
Yuming Wang 193304b51b [SPARK-27441][SQL][TEST] Add read/write tests to Hive serde tables
## What changes were proposed in this pull request?

The versions between Hive, Parquet and ORC after the built-in Hive upgraded to 2.3.5 for Hadoop 3.2:

- built-in Hive is 1.2.1.spark2:

  | ORC | Parquet
-- | -- | --
Spark datasource table | 1.5.5 | 1.10.1
Spark hive table | Hive built-in | 1.6.0
Apache Hive 1.2.1 | Hive built-in | 1.6.0

- built-in Hive is 2.3.5:

  | ORC | Parquet
-- | -- | --
Spark datasource table | 1.5.5 | 1.10.1
Spark hive table | 1.5.5 | [1.10.1](https://github.com/apache/spark/pull/24346)
Apache Hive 2.3.5 | 1.3.4 | 1.8.1

We should add a test for Hive Serde table. This pr adds tests to test read/write of all supported data types using Parquet and ORC.

## How was this patch tested?

unit tests

Closes #24345 from wangyum/SPARK-27441.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 08:35:58 -07:00
Yuming Wang dcacfc5da6 [SPARK-27074][SQL][test-hadoop3.2][test-maven] Hive 3.1 metastore support HiveClientImpl.runHive
## What changes were proposed in this pull request?

Hive 3.1.1's `CommandProcessor` have 2 changes:
1. [HIVE-17626](https://issues.apache.org/jira/browse/HIVE-17626)(Hive 3.0.0) add ReExecDriver. So the current code path is: 02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L736-L742)
We can disable `hive.query.reexecution.enabled` to workaround this change.
2. [HIVE-18238](http://issues.apache.org/jira/browse/HIVE-18238)(Hive 3.0.0) changed the `Driver.close()` function return type. We can workaround it by ` driver.getClass.getMethod("close").invoke(driver)`

So Hive 3.1 metastore could support `HiveClientImpl.runHive` after this pr.

## How was this patch tested?

unit tests

Closes #23992 from wangyum/SPARK-27074.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 08:24:41 -07:00
rrusso2007 ebd1431a5a [SPARK-27801][SQL] Improve performance of InMemoryFileIndex.listLeafFiles for HDFS directories with many files
## What changes were proposed in this pull request?

InMemoryFileIndex.listLeafFiles should use listLocatedStatus for DistributedFileSystem. DistributedFileSystem overrides the listLocatedStatus method in order to do it with 1 single namenode call thus saving thousands of calls to getBlockLocations.

Currently in InMemoryFileIndex, all directory listings are done using FileSystem.listStatus following by individual calls to FileSystem.getFileBlockLocations. This is painstakingly slow for folders that have large numbers of files because this process happens serially and parallelism is only applied at the folder level, not the file level.

FileSystem also provides another API listLocatedStatus which returns the LocatedFileStatus objects that already have the block locations. In FileSystem main class this just delegates to listStatus and getFileBlockLocations similarly to the way Spark does it. However when HDFS specifically is the backing file system, DistributedFileSystem overrides this method and simply makes one single call to the namenode to retrieve the directory listing with the block locations. This avoids potentially thousands or more calls to namenode and also is more consistent because files will either exist with locations or not exist instead of having the FileNotFoundException exception case.

For our example directory with 6500 files, the load time of spark.read.parquet was reduced 96x from 76 seconds to .8 seconds. This savings only goes up with the number of files in the directory.

In the pull request instead of using this method always which could lead to a FileNotFoundException that could be tough to decipher in the default FileSystem implementation, this method is only used when the FileSystem is a DistributedFileSystem and otherwise the old logic still applies.

## How was this patch tested?

test suite ran

Closes #24672 from rrusso2007/master.

Authored-by: rrusso2007 <rrusso2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-25 15:49:30 -07:00
Yesheng Ma 5e3520f7f4 [SPARK-27809][SQL] Make optional clauses order insensitive for CREATE DATABASE/VIEW SQL statement
## What changes were proposed in this pull request?

Each time, when I write a complex CREATE DATABASE/VIEW statements, I have to open the .g4 file to find the EXACT order of clauses in CREATE TABLE statement. When the order is not right, I will get A strange confusing error message generated from ANTLR4.

The original g4 grammar for CREATE VIEW is
```
CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [db_name.]view_name
  [(col_name1 [COMMENT col_comment1], ...)]
  [COMMENT table_comment]
  [TBLPROPERTIES (key1=val1, key2=val2, ...)]
AS select_statement
```
The proposal is to make the following clauses order insensitive.
```
  [COMMENT table_comment]
  [TBLPROPERTIES (key1=val1, key2=val2, ...)]
```
–
The original g4 grammar for CREATE DATABASE is
```
CREATE (DATABASE|SCHEMA) [IF NOT EXISTS] db_name
  [COMMENT comment_text]
  [LOCATION path]
  [WITH DBPROPERTIES (key1=val1, key2=val2, ...)]
```
The proposal is to make the following clauses order insensitive.
```
  [COMMENT comment_text]
  [LOCATION path]
  [WITH DBPROPERTIES (key1=val1, key2=val2, ...)]
```
## How was this patch tested?

By adding new unit tests to test duplicate clauses and modifying some existing unit tests to test whether those clauses are actually order insensitive

Closes #24681 from yeshengm/create-view-parser.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-24 15:19:14 -07:00
maryannxue de13f70ce1 [SPARK-27824][SQL] Make rule EliminateResolvedHint idempotent
## What changes were proposed in this pull request?

This fix prevents the rule EliminateResolvedHint from being applied again if it's already applied.

## How was this patch tested?

Added new UT.

Closes #24692 from maryannxue/eliminatehint-bug.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-24 11:25:22 -07:00
Wenchen Fan 7d318bfe90 [SPARK-26356][SQL] remove SaveMode from data source v2
## What changes were proposed in this pull request?

In data source v1, save mode specified in `DataFrameWriter` is passed to data source implementation directly, and each data source can define its own behavior about save mode. This is confusing and we want to get rid of save mode in data source v2.

For data source v2, we expect data source to implement the `TableCatalog` API, and end-users use SQL(or the new write API described in [this doc](https://docs.google.com/document/d/1gYm5Ji2Mge3QBdOliFV5gSPTKlX4q1DCBXIkiyMv62A/edit?ts=5ace0718#heading=h.e9v1af12g5zo)) to acess data sources. The SQL API has very clear semantic and we don't need save mode at all.

However, for simple data sources that do not have table management (like a JIRA data source, a noop sink, etc.), it's not ideal to ask them to implement the `TableCatalog` API, and throw exception here and there.

`TableProvider` API is created for simple data sources. It can only get tables, without any other table management methods. This means, it can only deal with existing tables.

`TableProvider` fits well with `DataStreamReader` and `DataStreamWriter`, as they can only read/write existing tables. However, `TableProvider` doesn't fit `DataFrameWriter` well, as the save mode requires more than just get table. More specifically, `ErrorIfExists` mode needs to check if table exists, and create table. `Ignore` mode needs to check if table exists. When end-users specify `ErrorIfExists` or `Ignore` mode and write data to `TableProvider` via `DataFrameWriter`, Spark fails the query and asks users to use `Append` or `Overwrite` mode.

The file source is in the middle of `TableProvider` and `TableCatalog`: it's simple but it can check table(path) exists and create table(path). That said, file source supports all the save modes.

Currently file source implements `TableProvider`, and it's not working because `TableProvider` doesn't support `ErrorIfExists` and `Ignore` modes. Ideally we should create a new API for path-based data sources, but to unblock the work of file source v2 migration, this PR proposes to special-case file source v2 in `DataFrameWriter`, to make it work.

This PR also removes `SaveMode` from data source v2, as now only the internal file source v2 needs it.

## How was this patch tested?

existing tests

Closes #24233 from cloud-fan/file.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-24 10:45:46 -07:00
Yuming Wang 24180c00e0 [SPARK-27831][SQL][TEST][test-hadoop3.2] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Closes #24695 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-24 10:33:34 -07:00
Ryan Blue 6b28497d6f [SPARK-27732][SQL] Add v2 CreateTable implementation.
## What changes were proposed in this pull request?

This adds a v2 implementation of create table:
* `CreateV2Table` is the logical plan, named using v2 to avoid conflicting with the existing plan
* `CreateTableExec` is the physical plan

## How was this patch tested?

Added resolution and v2 SQL tests.

Closes #24617 from rdblue/SPARK-27732-add-v2-create-table.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-24 11:13:22 +08:00
gatorsmile f94247ec90 [SPARK-27770][SQL][PART 1] Port AGGREGATES.sql
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. 02ddd49932/src/test/regress/sql/aggregates.sql (L1-L143)

The expected results can be found in the link: https://github.com/postgres/postgres/blob/master/src/test/regress/expected/aggregates.out

When porting the test cases, found three PostgreSQL specific features that do not exist in Spark SQL.
- https://issues.apache.org/jira/browse/SPARK-27765: Type Casts: expression::type
- https://issues.apache.org/jira/browse/SPARK-27766: Data type: POINT(x, y)
- https://issues.apache.org/jira/browse/SPARK-27767: Built-in function: generate_series

Also, found two bugs:
- https://issues.apache.org/jira/browse/SPARK-27768: Infinity, -Infinity, NaN should be recognized in a case insensitive manner
- https://issues.apache.org/jira/browse/SPARK-27769: Handling of sublinks within outer-level aggregates.

This PR also fixes the error message when the column can't be resolved.

For running the regression tests, this PR also added three tables `aggtest`, `onek` and `tenk1` from the postgreSQL data sets: 02ddd49932/src/test/regress/data

## How was this patch tested?
N/A

Closes #24640 from gatorsmile/addTestCase.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-23 16:34:37 -07:00
HyukjinKwon c1e555711b Revert "Revert "[SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values""
This reverts commit 855399bbad.
2019-05-24 05:36:17 +09:00
HyukjinKwon 1ba4011a7f Revert "Revert "[SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…""
This reverts commit 516b0fb537.
2019-05-24 05:36:08 +09:00
Wenchen Fan 1a68fc38f0 [SPARK-27816][SQL] make TreeNode tag type safe
## What changes were proposed in this pull request?

Add type parameter to `TreeNodeTag`.

## How was this patch tested?

existing tests

Closes #24687 from cloud-fan/tag.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-23 11:53:21 -07:00
HyukjinKwon 516b0fb537 Revert "[SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…"
This reverts commit 40668c53ed.
2019-05-24 03:17:06 +09:00
HyukjinKwon 855399bbad Revert "[SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values"
This reverts commit 42cb4a2ccd.
2019-05-24 03:16:24 +09:00
Wenchen Fan a590a935b1 [SPARK-27806][SQL] byName/byPosition should apply to struct fields as well
## What changes were proposed in this pull request?

When writing a query to data source v2, we have 2 modes to resolve the input query's output: byName or byPosition.

For byName mode, we would reorder the top level columns according to the name, and add type cast if possible. If the names don't match, we fail.

For byPosition mode, we don't do the reorder, and just add type cast directly if possible.

However, for struct type fields, we always apply byName mode. We should ignore the name difference if byPosition mode is used.

## How was this patch tested?

new tests

Closes #24678 from cloud-fan/write.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-23 10:37:45 -07:00
Yuming Wang 76988dd4a2 [SPARK-27737][FOLLOW-UP][SQL][test-hadoop3.2] Update Hive test jars from 2.3.4 to 2.3.5
## What changes were proposed in this pull request?

This pr update `hive-contrib-2.3.4.jar` to `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.4.jar` to `hive-hcatalog-core-2.3.5.jar`.

## How was this patch tested?

Existing test

Closes #24673 from wangyum/SPARK-27737-hive.jar.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-22 08:29:06 -07:00
Dongjoon Hyun a24cdc00bf [SPARK-27800][SQL][HOTFIX][FOLLOWUP] Fix wrong answer on BitwiseXor test cases
This PR is a follow up of https://github.com/apache/spark/pull/24669 to fix the wrong answers used in test cases.

Closes #24674 from dongjoon-hyun/SPARK-27800.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-22 03:11:29 -07:00
Liu Xiao bf617996aa [SPARK-27800][SQL][DOC] Fix wrong answer of example for BitwiseXor
## What changes were proposed in this pull request?

Fix example for bitwise xor function. 3 ^ 5 should be 6 rather than 2.
- See https://spark.apache.org/docs/latest/api/sql/index.html#_14

## How was this patch tested?

manual tests

Closes #24669 from alex-lx/master.

Authored-by: Liu Xiao <hhdxlx@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 21:52:19 -07:00
David Vogelbacher 034cb139a1 [SPARK-27778][PYTHON] Fix toPandas conversion of empty DataFrame with Arrow enabled
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/22275 introduced a performance improvement where we send partitions out of order to python and then, as a last step, send the partition order as well.
However, if there are no partitions we will never send the partition order and we will get an "EofError" on the python side.
This PR fixes this by also sending the partition order if there are no partitions present.

## How was this patch tested?
New unit test added.

Closes #24650 from dvogelbacher/dv/fixNoPartitionArrowConversion.

Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-22 13:21:26 +09:00
Wenchen Fan 03c9e8adee [SPARK-24586][SQL] Upcast should not allow casting from string to other types
## What changes were proposed in this pull request?

When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet.

However, the current upcast behavior is a little weird, we don't allow up casting from string to numeric, but allow non-numeric types as the target, like boolean, date, etc.

As a result, `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution.

The motivation of the up cast is to prevent things like runtime NPE, it's more reasonable to make up cast stricter.

This PR does 2 things:
1. rename `Cast.canSafeCast` to `Cast.canUpcast`, and support complex typres
2. remove `Cast.mayTruncate` and replace it with `!Cast.canUpcast`

Note that, the up cast change also affects persistent view resolution. But since we don't support changing column types of an existing table, there is no behavior change here.

## How was this patch tested?

new tests

Closes #21586 from cloud-fan/cast.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-22 11:35:51 +08:00
Gengliang Wang c3c443ca8c [SPARK-27698][SQL] Add new method convertibleFilters for getting pushed down filters in Parquet file reader
## What changes were proposed in this pull request?

To return accurate pushed filters in Parquet file scan(https://github.com/apache/spark/pull/24327#pullrequestreview-234775673), we can process the original data source filters in the following way:
1. For "And" operators, split the conjunctive predicates and try converting each of them. After that
1.1 if partially predicate pushed down is allowed, return convertible results;
1.2 otherwise, return the whole predicate if convertible, or empty result if not convertible.

2. For "Or" operators, if both children can be  pushed down, it is partially or totally convertible; otherwise, return empty result

3. For other operators, they are not able to be partially pushed down.
2.1 if the entire predicate is convertible, return itself
2.2 otherwise, return an empty result.

This PR also contains code refactoring. Currently `ParquetFilters. createFilter ` accepts parameter `schema: MessageType` and create field mapping for every input filter. We can make it a class member and avoid creating the `nameToParquetField` mapping for every input filter.

## How was this patch tested?

Unit test

Closes #24597 from gengliangwang/refactorParquetFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-22 11:27:25 +08:00
Yuming Wang 6cd1efd0ae [SPARK-27737][SQL] Upgrade to Hive 2.3.5 for Hive Metastore Client and Hadoop-3.2 profile
## What changes were proposed in this pull request?

This PR aims to upgrade to Hive 2.3.5 for Hive Metastore Client and Hadoop-3.2 profile.

Release Notes - Hive - Version 2.3.5

- [[HIVE-21536](https://issues.apache.org/jira/browse/HIVE-21536)] - Backport HIVE-17764 to branch-2.3
- [[HIVE-21585](https://issues.apache.org/jira/browse/HIVE-21585)] - Upgrade branch-2.3 to ORC 1.3.4
- [[HIVE-21639](https://issues.apache.org/jira/browse/HIVE-21639)] - Spark test failed since HIVE-10632
- [[HIVE-21680](https://issues.apache.org/jira/browse/HIVE-21680)] - Backport HIVE-17644 to branch-2 and branch-2.3

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12345394&styleName=Text&projectId=12310843

## How was this patch tested?

This PR is tested in two ways.
- Pass the Jenkins with the default configuration for `Hive Metastore Client` testing.
- Pass the Jenkins with `test-hadoop3.2` configuration for `Hadoop 3.2` testing.

Closes #24620 from wangyum/SPARK-27737.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-22 10:24:17 +09:00
williamwong 8442d94fb1 [SPARK-27248][SQL] refreshTable should recreate cache with same cache name and storage level
If we refresh a cached table, the table cache will be first uncached and then recache (lazily). Currently, the logic is embedded in CatalogImpl.refreshTable method.
The current implementation does not preserve the cache name and storage level. As a result, cache name and cache level could be changed after a REFERSH. IMHO, it is not what a user would expect.
I would like to fix this behavior by first save the cache name and storage level for recaching the table.

Two unit tests are added to make sure cache name is unchanged upon table refresh. Before applying this patch, the test created for qualified case would fail.

Closes #24221 from William1104/feature/SPARK-27248.

Lead-authored-by: williamwong <william1104@gmail.com>
Co-authored-by: William Wong <william1104@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 11:37:16 -07:00
Liang-Chi Hsieh c033a3e1e6 [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans
## What changes were proposed in this pull request?

Because a temporary view is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.

```scala
scala> spark.range(10).createOrReplaceTempView("test")
scala> spark.range(5).createOrReplaceTempView("test2")
scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
scala> val df = spark.sql("select * from tmp001")
scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
scala> df.show
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
scala> df.explain(true)
```

Before:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#2L]
+- SubqueryAlias `tmp001`
   +- Project [id#2L]
      +- SubqueryAlias `test2`
         +- Range (0, 5, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 5, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 5, step=1, splits=12)
```

After:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#0L]
+- SubqueryAlias `tmp001`
   +- Project [id#0L]
      +- SubqueryAlias `test`
         +- Range (0, 10, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 10, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 10, step=1, splits=12)
```

Previous PR to this issue has a regression when to explain an explain statement, like `sql("explain select 1").explain(true)`. This new fix is following up with hvanhovell's advice at https://github.com/apache/spark/pull/24464#issuecomment-494165538.

Explain an explain:
```scala
scala> sql("explain select 1").explain(true)
== Parsed Logical Plan ==
ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false

== Analyzed Logical Plan ==
plan: string
ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false

== Optimized Logical Plan ==
ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false

== Physical Plan ==
Execute ExplainCommand
   +- ExplainCommand 'Project [unresolvedalias(1, None)], false, false, false
```

Btw, I found there is a regression after applying hvanhovell's advice:

```scala
spark.readStream
      .format("org.apache.spark.sql.streaming.test")
      .load()
      .explain(true)
```

```scala
== Parsed Logical Plan ==
StreamingRelation DataSource(org.apache.spark.sql.test.TestSparkSession3e8c7175,org.apache.spark.sql.streaming.test,List(),None,List(),None,Map(),None
), dummySource, [a#559]

== Analyzed Logical Plan ==
a: int
StreamingRelation DataSource(org.apache.spark.sql.test.TestSparkSession3e8c7175,org.apache.spark.sql.streaming.test,List(),None,List(),None,Map(),Non$
), dummySource, [a#559]

== Optimized Logical Plan ==
org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();;
dummySource
== Physical Plan ==
org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();;
dummySource
```

So I did a change to that to fix it too.

## How was this patch tested?

Added test and manually test.

Closes #24654 from viirya/SPARK-27439-3.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-21 11:27:05 -07:00
Wenchen Fan 1e0facb60d [SQL][DOC][MINOR] update documents for Table and WriteBuilder
## What changes were proposed in this pull request?

Update the docs to reflect the changes made by https://github.com/apache/spark/pull/24129

## How was this patch tested?

N/A

Closes #24658 from cloud-fan/comment.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 09:29:06 -07:00
Josh Rosen 604aa1b045 [SPARK-27786][SQL] Fix Sha1, Md5, and Base64 codegen when commons-codec is shaded
## What changes were proposed in this pull request?

When running a custom build of Spark which shades `commons-codec`, the `Sha1` expression generates code which fails to compile:

```
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, Column 93: A method named "sha1Hex" is not declared in any enclosing class nor any supertype, nor through a static import
```

This is caused by an interaction between Spark's code generator and the shading: the current codegen template includes the string `org.apache.commons.codec.digest.DigestUtils.sha1Hex` as part of a larger string literal, preventing JarJarLinks from being able to replace the class name with the shaded class's name. As a result, the generated code still references the original unshaded class name name, triggering an error in case the original unshaded dependency isn't on the path.

This problem impacts the `Sha1`, `Md5`, and `Base64` expressions.

To fix this problem and allow for proper shading, this PR updates the codegen templates to replace the hardcoded class names with `${classof[<name>].getName}` calls.

## How was this patch tested?

Existing tests.

To ensure that I found all occurrences of this problem, I used IntelliJ's "Find in Path" to search for lines matching the regex `^(?!import|package).*(org|com|net|io)\.(?!apache\.spark)` and then filtered matches to inspect only non-test "Usage in string constants" cases. This isn't _perfect_ but I think it'll catch most cases.

Closes #24655 from JoshRosen/fix-shaded-apache-commons.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-21 21:18:34 +08:00
Dongjoon Hyun 039db879f4 Revert "[SPARK-27439][SQL] Explainging Dataset should show correct resolved plans"
This reverts commit 4b725e50a7.
2019-05-20 15:07:00 -07:00
Wenchen Fan 0e6601acdf [SPARK-27747][SQL] add a logical plan link in the physical plan
## What changes were proposed in this pull request?

It's pretty useful if we can convert a physical plan back to a logical plan, e.g., in https://github.com/apache/spark/pull/24389

This PR introduces a new feature to `TreeNode`, which allows `TreeNode` to carry some extra information via a mutable map, and keep the information when it's copied.

The planner leverages this feature to put the logical plan into the physical plan.

## How was this patch tested?

a test suite that runs all TPCDS queries and checks that some common physical plans contain the corresponding logical plans.

Closes #24626 from cloud-fan/link.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Peng Bo <bo.peng1019@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-20 13:42:25 -07:00
Yuming Wang 5dda1fe296 [SPARK-27699][FOLLOW-UP][SQL][test-hadoop3.2][test-maven] Fix hadoop-3.2 test error
## What changes were proposed in this pull request?

This pr fix `hadoop-3.2` test error:
```
- SPARK-27699 Converting disjunctions into ORC SearchArguments *** FAILED ***
  Expected "...SS_THAN_EQUALS a 10)[
  leaf-1 = (LESS_THAN a 1)
  ]expr = (or (not leaf...", but got "...SS_THAN_EQUALS a 10)[, leaf-1 = (LESS_THAN a 1), ]expr = (or (not leaf..." (HiveOrcFilterSuite.scala:445)
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105514/consoleFull

## How was this patch tested?

N/A

Closes #24639 from wangyum/SPARK-27699.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-20 13:04:05 -07:00
Yuming Wang 974b879220 [SPARK-27694][SQL] Support auto-updating table statistics for data source CTAS command
## What changes were proposed in this pull request?

This pr makes it support collect statistics when CTAS(create a data source table using the result of a query).

## How was this patch tested?

unit tests and manual tests:
```sql
bin/spark-sql --conf spark.sql.statistics.size.autoUpdate.enabled=true -S

spark-sql> CREATE TABLE spark_27694 USING parquet AS SELECT 'a', 'b';
spark-sql> DESC FORMATTED spark_27694;
a	string	NULL
b	string	NULL

# Detailed Table Information
Database	default
Table	spark_27694
Owner	root
Created Time	Mon May 13 19:45:33 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	parquet
Statistics	561 bytes
Location	file:/user/hive/warehouse/spark_27694
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
```

Closes #24596 from wangyum/SPARK-27694.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 22:29:40 -07:00
Ryan Blue bc46feaced [SPARK-27693][SQL] Add default catalog property
Add a SQL config property for the default v2 catalog.

Existing tests for regressions.

Closes #24594 from rdblue/SPARK-27693-add-default-catalog-config.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 21:30:52 -07:00
HyukjinKwon 2431ab0999 [SPARK-27771][SQL] Add SQL description for grouping functions (cube, rollup, grouping and grouping_id)
## What changes were proposed in this pull request?

Both look added as of 2.0 (see SPARK-12541 and SPARK-12706). I referred existing docs and examples in other API docs.

## How was this patch tested?

Manually built the documentation and, by running examples, by running `DESCRIBE FUNCTION EXTENDED`.

Closes #24642 from HyukjinKwon/SPARK-27771.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 19:26:20 -07:00
liuxian 9bca99b29b [SPARK-27552][SQL] The configuration hive.exec.stagingdir is invalid on Windows OS
## What changes were proposed in this pull requesst?
If we set `hive.exec.stagingdir=.test-staging\tmp`,
But the staging directory is still `.hive-staging` on Windows OS.

Reasons for failure:
Test code:
```
 val path = new Path("C:\\test\\hivetable")
  println("path.toString: " + path.toString)
  println("path.toUri.getPath: " + path.toUri.getPath)
```

Output:
```
path.toString: C:/test/hivetable
path.toUri.getPath: /C:/test/hivetable
```
 We can see that `path.toUri.getPath` has one more separator than `path.toString`,  and the separator is   ' / ',  not  ' \ '
So `stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")` will return false
## How was this patch tested?
1. Existed tests
2. Manual testing on Windows OS

Closes #24446 from 10110346/stagingdir.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-17 14:00:17 -05:00
Gengliang Wang e39e97b73a [SPARK-27699][SQL] Partially push down disjunctive predicated in Parquet/ORC
## What changes were proposed in this pull request?

Currently, in `ParquetFilters` and `OrcFilters`, if the child predicate of `Or` operator can't be entirely pushed down, the predicates will be thrown away.
In fact, the conjunctive predicates under `Or` operators can be partially pushed down.
For example, says `a` and `b` are convertible, while `c` can't be pushed down, the predicate
`a or (b and c)`
can be converted as
`(a or b) and (a or c)`
We can still push down `(a or b)`.
We can't push down disjunctive predicates only when one of its children is not partially convertible.

This PR also improve the filter pushing down logic in `DataSourceV2Strategy`. With partial filter push down in `Or` operator, the result of `pushedFilters()` might not exist in the mapping `translatedFilterToExpr`.  To fix it, this PR changes the mapping `translatedFilterToExpr` as leaf filter expression to `sources.filter`, and later on rebuild the whole expression with the mapping.
## How was this patch tested?

Unit test

Closes #24598 from gengliangwang/pushdownDisjunctivePredicates.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-17 19:25:24 +08:00
Wenchen Fan fc5bd6da77 [SPARK-27576][SQL] table capability to skip the output column resolution
## What changes were proposed in this pull request?

Currently we have an analyzer rule, which resolves the output columns of data source v2 writing plans, to make sure the schema of input query is compatible with the table.

However, not all data sources need this check. For example, the `NoopDataSource` doesn't care about the schema of input query at all.

This PR introduces a new table capability: ACCEPT_ANY_SCHEMA. If a table reports this capability, we skip resolving output columns for it during write.

Note that, we already skip resolving output columns for `NoopDataSource` because it implements `SupportsSaveMode`. However, `SupportsSaveMode` is a hack and will be removed soon.

## How was this patch tested?

new test cases

Closes #24469 from cloud-fan/schema-check.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-16 16:24:53 -07:00
Shixiong Zhu 6a317c8f01 [SPARK-27735][SS] Parsing interval string should be case-insensitive in SS
## What changes were proposed in this pull request?

Some APIs in Structured Streaming requires the user to specify an interval. Right now these APIs don't accept upper-case strings.

This PR adds a new method `fromCaseInsensitiveString` to `CalendarInterval` to support paring upper-case strings, and fixes all APIs that need to parse an interval string.

## How was this patch tested?

The new unit test.

Closes #24619 from zsxwing/SPARK-27735.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-16 13:58:27 -07:00
shivusondur c6a45e6f67 [SPARK-27722][SQL] removed the unsed "UnsafeKeyValueSorter" file.
## What changes were proposed in this pull request?

removed the unused "UnsafeKeyValueSorter.java" file

## How was this patch tested?

Ran Compilation and UT locally.

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

Closes #24622 from shivusondur/jira27722.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-16 18:22:06 +08:00
Wenchen Fan 3e30a98810 [SPARK-27674][SQL] the hint should not be dropped after cache lookup
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20365 .

#20365 fixed this problem when the hint node is a root node. This PR fixes this problem for all the cases.

## How was this patch tested?

a new test

Closes #24580 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 15:47:52 -07:00
Yuming Wang 02c33694c8 [SPARK-27354][SQL] Move incompatible code from the hive-thriftserver module to sql/hive-thriftserver/v1.2.1
## What changes were proposed in this pull request?

When we upgraded the built-in Hive to 2.3.4, the current `hive-thriftserver` module is not compatible, such as these Hive changes:
1. [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442) HiveServer2: Refactor/repackage HiveServer2's Thrift code so that it can be used in the tasks
2. [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237) Use slf4j as logging facade
3. [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) HiveServer2: Support delegation token based connection when using http transport

So this PR moves the incompatible code to `sql/hive-thriftserver/v1.2.1` and copies it to `sql/hive-thriftserver/v2.3.4` for the next code review.

## How was this patch tested?

manual tests:
```
diff -urNa sql/hive-thriftserver/v1.2.1 sql/hive-thriftserver/v2.3.4
```

Closes #24282 from wangyum/SPARK-27354.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 14:52:08 -07:00
Xingbo Jiang 0bba5cf568 [SPARK-20774][SPARK-27036][SQL] Cancel the running broadcast execution on BroadcastTimeout
## What changes were proposed in this pull request?

In the existing code, a broadcast execution timeout for the Future only causes a query failure, but the job running with the broadcast and the computation in the Future are not canceled. This wastes resources and slows down the other jobs. This PR tries to cancel both the running job and the running hashed relation construction thread.

## How was this patch tested?

Add new test suite `BroadcastExchangeExec`

Closes #24595 from jiangxb1987/SPARK-20774.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 14:47:15 -07:00
Sean Owen bfb3ffe9b3 [SPARK-27682][CORE][GRAPHX][MLLIB] Replace use of collections and methods that will be removed in Scala 2.13 with work-alikes
## What changes were proposed in this pull request?

This replaces use of collection classes like `MutableList` and `ArrayStack` with workalikes that are available in 2.12, as they will be removed in 2.13. It also removes use of `.to[Collection]` as its uses was superfluous anyway. Removing `collection.breakOut` will have to wait until 2.13

## How was this patch tested?

Existing tests

Closes #24586 from srowen/SPARK-27682.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-15 09:29:12 -05:00
xy_xin fd9acf23b0 [SPARK-27713][SQL] Move org.apache.spark.sql.execution.* in catalyst to core
## What changes were proposed in this pull request?

`RecordBinaryComparator`, `UnsafeExternalRowSorter` and `UnsafeKeyValueSorter` now locates in catalyst, which should be moved to core, as they're used only in physical plan.

## How was this patch tested?

exist tests.

Closes #24607 from xianyinxin/SPARK-27713.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-15 15:24:21 +08:00
Ryan Blue 2da5b21834 [SPARK-24923][SQL] Implement v2 CreateTableAsSelect
## What changes were proposed in this pull request?

This adds a v2 implementation for CTAS queries

* Update the SQL parser to parse CREATE queries using multi-part identifiers
* Update `CheckAnalysis` to validate partitioning references with the CTAS query schema
* Add `CreateTableAsSelect` v2 logical plan and `CreateTableAsSelectExec` v2 physical plan
* Update create conversion from `CreateTableAsSelectStatement` to support the new v2 logical plan
* Update `DataSourceV2Strategy` to convert v2 CTAS logical plan to the new physical plan
* Add `findNestedField` to `StructType` to support reference validation

## How was this patch tested?

We have been running these changes in production for several months. Also:

* Add a test suite `CreateTablePartitioningValidationSuite` for new analysis checks
* Add a test suite for v2 SQL, `DataSourceV2SQLSuite`
* Update catalyst `DDLParserSuite` to use multi-part identifiers (`Seq[String]`)
* Add test cases to `PlanResolutionSuite` for v2 CTAS: known catalog and v2 source implementation

Closes #24570 from rdblue/SPARK-24923-add-v2-ctas.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-15 11:24:03 +08:00
Yuming Wang fee695d0cf [SPARK-27690][SQL] Remove materialized views first in HiveClientImpl.reset
## What changes were proposed in this pull request?

We should remove materialized view first otherwise(note that Hive 3.1 could reproduce this issue):
```scala
Cause: org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: DELETE on table 'TBLS' caused a violation of foreign key constraint 'MV_TABLES_USED_FK2' for key (4).  The statement has been rolled back.
at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedConnection.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.ConnectionChild.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedStatement.executeStatement(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedPreparedStatement.executeBatchElement(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedStatement.executeLargeBatch(Unknown Source)
```

## How was this patch tested?

Existing test

Closes #24592 from wangyum/SPARK-27690.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-14 09:05:22 -07:00
Sean Owen a10608cb82 [SPARK-27680][CORE][SQL][GRAPHX] Remove usage of Traversable
## What changes were proposed in this pull request?

This removes usage of `Traversable`, which is removed in Scala 2.13. This is mostly an internal change, except for the change in the `SparkConf.setAll` method. See additional comments below.

## How was this patch tested?

Existing tests.

Closes #24584 from srowen/SPARK-27680.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-14 09:14:56 -05:00
mingbo.pb 66f5a42ca5 [SPARK-27638][SQL] Cast string to date/timestamp in binary comparisons with dates/timestamps
## What changes were proposed in this pull request?

The below example works with both Mysql and Hive, however not with spark.

```
mysql> select * from date_test where date_col >= '2000-1-1';
+------------+
| date_col   |
+------------+
| 2000-01-01 |
+------------+
```
The reason is that Spark casts both sides to String type during date and string comparison for partial date support. Please find more details in https://issues.apache.org/jira/browse/SPARK-8420.

Based on some tests, the behavior of Date and String comparison in Hive and Mysql:
Hive: Cast to Date, partial date is not supported
Mysql: Cast to Date, certain "partial date" is supported by defining certain date string parse rules. Check out str_to_datetime in https://github.com/mysql/mysql-server/blob/5.5/sql-common/my_time.c

As below date patterns have been supported, the PR is to cast string to date when comparing string and date:
```
`yyyy`
`yyyy-[m]m`
`yyyy-[m]m-[d]d`
`yyyy-[m]m-[d]d `
`yyyy-[m]m-[d]d *`
`yyyy-[m]m-[d]dT*
```

## How was this patch tested?
UT has been added

Closes #24567 from pengbo/SPARK-27638.

Authored-by: mingbo.pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-14 17:10:36 +08:00
Liang-Chi Hsieh 8b0bdaa8e0 [SPARK-27671][SQL] Fix error when casting from a nested null in a struct
## What changes were proposed in this pull request?

When a null in a nested field in struct, casting from the struct throws error, currently.

```scala
scala> sql("select cast(struct(1, null) as struct<a:int,b:int>)").show
scala.MatchError: NullType (of class org.apache.spark.sql.types.NullType$)
  at org.apache.spark.sql.catalyst.expressions.Cast.castToInt(Cast.scala:447)
  at org.apache.spark.sql.catalyst.expressions.Cast.cast(Cast.scala:635)
  at org.apache.spark.sql.catalyst.expressions.Cast.$anonfun$castStruct$1(Cast.scala:603)
```

Similarly, inline table, which casts null in nested field under the hood, also throws an error.

```scala
scala> sql("select * FROM VALUES (('a', (10, null))), (('b', (10, 50))), (('c', null)) AS tab(x, y)").show
org.apache.spark.sql.AnalysisException: failed to evaluate expression named_struct('col1', 10, 'col2', NULL): NullType (of class org.apache.spark.sql.t
ypes.NullType$); line 1 pos 14
  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:47)
  at org.apache.spark.sql.catalyst.analysis.ResolveInlineTables.$anonfun$convert$6(ResolveInlineTables.scala:106)
```

This fixes the issue.

## How was this patch tested?

Added tests.

Closes #24576 from viirya/cast-null.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-13 12:40:46 -07:00
Yuming Wang f3ddd6f9da [SPARK-27402][SQL][TEST-HADOOP3.2][TEST-MAVEN] Fix hadoop-3.2 test issue(except the hive-thriftserver module)
## What changes were proposed in this pull request?

This pr fix hadoop-3.2 test issues(except the `hive-thriftserver` module):
1. Add `hive.metastore.schema.verification` and `datanucleus.schema.autoCreateAll` to HiveConf.
2. hadoop-3.2 support access the Hive metastore from 0.12 to 2.2

After [SPARK-27176](https://issues.apache.org/jira/browse/SPARK-27176) and this PR, we upgraded the built-in Hive to 2.3 when enabling the Hadoop 3.2+ profile. This upgrade fixes the following issues:
- [HIVE-6727](https://issues.apache.org/jira/browse/HIVE-6727): Table level stats for external tables are set incorrectly.
- [HIVE-15653](https://issues.apache.org/jira/browse/HIVE-15653): Some ALTER TABLE commands drop table stats.
- [SPARK-12014](https://issues.apache.org/jira/browse/SPARK-12014): Spark SQL query containing semicolon is broken in Beeline.
- [SPARK-25193](https://issues.apache.org/jira/browse/SPARK-25193): insert overwrite doesn't throw exception when drop old data fails.
- [SPARK-25919](https://issues.apache.org/jira/browse/SPARK-25919): Date value corrupts when tables are "ParquetHiveSerDe" formatted and target table is Partitioned.
- [SPARK-26332](https://issues.apache.org/jira/browse/SPARK-26332): Spark sql write orc table on viewFS throws exception.
- [SPARK-26437](https://issues.apache.org/jira/browse/SPARK-26437): Decimal data becomes bigint to query, unable to query.

## How was this patch tested?
This pr test Spark’s Hadoop 3.2 profile on jenkins and #24591 test Spark’s Hadoop 2.7 profile on jenkins

This PR close #24591

Closes #24391 from wangyum/SPARK-27402.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-13 10:35:26 -07:00
Liang-Chi Hsieh d169b0aac3 [SPARK-27653][SQL] Add max_by() and min_by() SQL aggregate functions
## What changes were proposed in this pull request?

This PR goes to add `max_by()` and `min_by()` SQL aggregate functions.

Quoting from the [Presto docs](https://prestodb.github.io/docs/current/functions/aggregate.html#max_by)

> max_by(x, y) → [same as x]
> Returns the value of x associated with the maximum value of y over all input values.

`min_by()` works similarly.

## How was this patch tested?

Added tests.

Closes #24557 from viirya/SPARK-27653.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-13 22:37:34 +08:00
zhoukang 126310ca68 [SPARK-26601][SQL] Make broadcast-exchange thread pool configurable
## What changes were proposed in this pull request?

Currently,thread number of broadcast-exchange thread pool is fixed and keepAliveSeconds is also fixed as 60s.

```
object BroadcastExchangeExec {
  private[execution] val executionContext = ExecutionContext.fromExecutorService(
    ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128))
}

 /**
   * Create a cached thread pool whose max number of threads is `maxThreadNumber`. Thread names
   * are formatted as prefix-ID, where ID is a unique, sequentially assigned integer.
   */
  def newDaemonCachedThreadPool(
      prefix: String, maxThreadNumber: Int, keepAliveSeconds: Int = 60): ThreadPoolExecutor = {
    val threadFactory = namedThreadFactory(prefix)
    val threadPool = new ThreadPoolExecutor(
      maxThreadNumber, // corePoolSize: the max number of threads to create before queuing the tasks
      maxThreadNumber, // maximumPoolSize: because we use LinkedBlockingDeque, this one is not used
      keepAliveSeconds,
      TimeUnit.SECONDS,
      new LinkedBlockingQueue[Runnable],
      threadFactory)
    threadPool.allowCoreThreadTimeOut(true)
    threadPool
  }
```

But some times, if the Thead object do not GC quickly it may caused server(driver) OOM. In such case,we need to make this thread pool configurable.
A case has described in https://issues.apache.org/jira/browse/SPARK-26601

## How was this patch tested?
UT

Closes #23670 from caneGuy/zhoukang/make-broadcat-config.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-13 20:40:21 +09:00
Gengliang Wang be6d39c379 [SPARK-27668][SQL] File source V2: support reporting statistics
## What changes were proposed in this pull request?

In File source V1, the statistics of `HadoopFsRelation` is `compressionFactor * sizeInBytesOfAllFiles`.
To follow it, we can implement the interface SupportsReportStatistics in FileScan and report the same statistics.

## How was this patch tested?

Unit test

Closes #24571 from gengliangwang/stats.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-13 14:16:11 +08:00
Wenchen Fan 9ff77b198e [SPARK-27675][SQL] do not use MutableColumnarRow in ColumnarBatch
## What changes were proposed in this pull request?

To move DS v2 API to the catalyst module, we can't refer to an internal class (`MutableColumnarRow`) in `ColumnarBatch`.

This PR creates a read-only version of `MutableColumnarRow`, and use it in `ColumnarBatch`.

close https://github.com/apache/spark/pull/24546

## How was this patch tested?

existing tests

Closes #24581 from cloud-fan/mutable-row.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-12 19:59:56 +09:00
HyukjinKwon c71f217de1 [SPARK-27673][SQL] Add since info to random, regex, null expressions
## What changes were proposed in this pull request?

We should add since info to all expressions.

SPARK-7886 Rand / Randn
af3746ce0d RLike, Like (I manually checked that it exists from 1.0.0)
SPARK-8262 Split
SPARK-8256 RegExpReplace
SPARK-8255 RegExpExtract
9aadcffabd Coalesce / IsNull / IsNotNull (I manually checked that it exists from 1.0.0)
SPARK-14541 IfNull / NullIf / Nvl / Nvl2
SPARK-9080 IsNaN
SPARK-9168 NaNvl

## How was this patch tested?

N/A

Closes #24579 from HyukjinKwon/SPARK-27673.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-10 09:24:04 -07:00
HyukjinKwon 3442fcaa9b [SPARK-27672][SQL] Add since info to string expressions
## What changes were proposed in this pull request?

This PR adds since information to the all string expressions below:

SPARK-8241 ConcatWs
SPARK-16276 Elt
SPARK-1995 Upper / Lower
SPARK-20750 StringReplace
SPARK-8266 StringTranslate
SPARK-8244 FindInSet
SPARK-8253 StringTrimLeft
SPARK-8260 StringTrimRight
SPARK-8267 StringTrim
SPARK-8247 StringInstr
SPARK-8264 SubstringIndex
SPARK-8249 StringLocate
SPARK-8252 StringLPad
SPARK-8259 StringRPad
SPARK-16281 ParseUrl
SPARK-9154 FormatString
SPARK-8269 Initcap
SPARK-8257 StringRepeat
SPARK-8261 StringSpace
SPARK-8263 Substring
SPARK-21007 Right
SPARK-21007 Left
SPARK-8248 Length
SPARK-20749 BitLength
SPARK-20749 OctetLength
SPARK-8270 Levenshtein
SPARK-8271 SoundEx
SPARK-8238 Ascii
SPARK-20748 Chr
SPARK-8239 Base64
SPARK-8268 UnBase64
SPARK-8242 Decode
SPARK-8243 Encode
SPARK-8245 format_number
SPARK-16285 Sentences

## How was this patch tested?

N/A

Closes #24578 from HyukjinKwon/SPARK-27672.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-10 09:11:12 -07:00
Eric Liang 80de449f51 [MINOR][TEST] Fix schema mismatch error
## What changes were proposed in this pull request?

- the accumulator warning is too verbose
- when a test fails with schema mismatch, you never see the error message / exception

Closes #24549 from ericl/test-nits.

Lead-authored-by: Eric Liang <ekl@databricks.com>
Co-authored-by: Eric Liang <ekhliang@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-10 23:08:02 +08:00
Marco Gaido 78748b5752 [SPARK-27625][SQL] ScalaReflection support for annotated types
## What changes were proposed in this pull request?

If a type is annotated, `ScalaReflection` can fail if the datatype is an `Option`, a `Seq`, a `Map` and other similar types. This is because it assumes we are dealing with `TypeRef`, while types with annotations are `AnnotatedType`.

The PR deals with the case the annotation is present.

## How was this patch tested?

added UT

Closes #24564 from mgaido91/SPARK-27625.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-10 22:48:36 +08:00
Yuming Wang b5ffec12eb [SPARK-27563][FOLLOWUP] Fix to download new release from dist.apache.org
## What changes were proposed in this pull request?

`https://archive.apache.org/dist/spark/` does not have latest Spark 2.4.3:
<img src="https://user-images.githubusercontent.com/5399861/57288553-4264b600-70ec-11e9-8dcc-71b7589f5ad0.png" width="400">

This pr add `https://dist.apache.org/repos/dist/release/spark/` to mirrors list to download latest Spark.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105202/testReport/org.apache.spark.sql.hive/HiveExternalCatalogVersionsSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/

## How was this patch tested?

manual tests:
```
build/sbt "hive/testOnly *.HiveExternalCatalogVersionsSuite"  -Phive
```

Closes #24544 from wangyum/Unable-to-download-Spark-2.4.3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-09 08:45:52 -05:00
pgandhi 0969d7aa0c [SPARK-27207][SQL] : Ensure aggregate buffers are initialized again for So…
…rtBasedAggregate

Normally, the aggregate operations that are invoked for an aggregation buffer for User Defined Aggregate Functions(UDAF) follow the order like initialize(), update(), eval() OR initialize(), merge(), eval(). However, after a certain threshold configurable by spark.sql.objectHashAggregate.sortBased.fallbackThreshold is reached, ObjectHashAggregate falls back to SortBasedAggregator which invokes the merge or update operation without calling initialize() on the aggregate buffer.

## What changes were proposed in this pull request?

The fix here is to initialize aggregate buffers again when fallback to SortBasedAggregate operator happens.

## How was this patch tested?

The patch was tested as part of [SPARK-24935](https://issues.apache.org/jira/browse/SPARK-24935) as documented in PR https://github.com/apache/spark/pull/23778.

Closes #24149 from pgandhi999/SPARK-27207.

Authored-by: pgandhi <pgandhi@verizonmedia.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-09 11:12:20 +08:00
Gengliang Wang 78a403fab9 [SPARK-27627][SQL] Make option "pathGlobFilter" as a general option for all file sources
## What changes were proposed in this pull request?

### Background:
The data source option `pathGlobFilter` is introduced for Binary file format: https://github.com/apache/spark/pull/24354 , which can be used for filtering file names, e.g. reading `.png` files only while there is `.json` files in the same directory.

### Proposal:
Make the option `pathGlobFilter` as a general option for all file sources. The path filtering should happen in the path globbing on Driver.

### Motivation:
Filtering the file path names in file scan tasks on executors is kind of ugly.

### Impact:
1. The splitting of file partitions will be more balanced.
2. The metrics of file scan will be more accurate.
3. Users can use the option for reading other file sources.

## How was this patch tested?

Unit tests

Closes #24518 from gengliangwang/globFilter.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-09 08:41:43 +09:00
gengjiaan 57450ed9b7 [MINOR][SS] Rename secondLatestBatchId to secondLatestOffsets
## What changes were proposed in this pull request?

The mothod `populateStartOffsets` exists a inappropriate identifier `secondLatestBatchId`.
I think `secondLatestBatchId = latestBatchId - 1` and `offsetLog.get(latestBatchId - 1)` is a offset.
So I change the identifier as follows:
`secondLatestOffsets = offsetLog.get(latestBatchId - 1)`

## How was this patch tested?

Exists UT.

Closes #24550 from beliefer/fix-inappropriate-identifier.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-08 11:01:26 -07:00
Wenchen Fan bae5baae52 [SPARK-27642][SS] make v1 offset extends v2 offset
## What changes were proposed in this pull request?

To move DS v2 to the catalyst module, we can't make v2 offset rely on v1 offset, as v1 offset is in sql/core.

## How was this patch tested?

existing tests

Closes #24538 from cloud-fan/offset.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-07 23:03:15 -07:00
Yuming Wang 3ea44e52e7 [SPARK-27639][SQL] InMemoryTableScan shows the table name on UI if possible
## What changes were proposed in this pull request?
<img src="https://user-images.githubusercontent.com/5399861/57213799-7bccf100-701a-11e9-9872-d90b4a185dc6.png" width="200">

It only shows `InMemoryTableScan` when scanning InMemoryTable.
When there are many InMemoryTables, it is difficult to distinguish which one is what we are looking for. This PR show the table name when scanning InMemoryTable.

## How was this patch tested?

unit tests and manual tests

After this PR:
<img src="https://user-images.githubusercontent.com/5399861/57269120-d3219e80-70b8-11e9-9e56-1b5d4c071660.png" width="200">

Closes #24534 from wangyum/SPARK-27639.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 21:00:13 -07:00
Jose Torres 83f628b57d [SPARK-27253][SQL][FOLLOW-UP] Add a legacy flag to restore old session init behavior
## What changes were proposed in this pull request?

Add a legacy flag to restore the old session init behavior, where SparkConf defaults take precedence over configs in a parent session.

Closes #24540 from jose-torres/oss.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 20:04:09 -07:00
Ryan Blue 303ee3fce0 [SPARK-24252][SQL] Add TableCatalog API
## What changes were proposed in this pull request?

This adds the TableCatalog API proposed in the [Table Metadata API SPIP](https://docs.google.com/document/d/1zLFiA1VuaWeVxeTDXNg8bL6GP3BVoOZBkewFtEnjEoo/edit#heading=h.m45webtwxf2d).

For `TableCatalog` to use `Table`, it needed to be moved into the catalyst module where the v2 catalog API is located. This also required moving `TableCapability`. Most of the files touched by this PR are import changes needed by this move.

## How was this patch tested?

This adds a test implementation and contract tests.

Closes #24246 from rdblue/SPARK-24252-add-table-catalog-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-08 10:31:06 +08:00
Adi Muraru 8ef4da753d [SPARK-27610][YARN] Shade netty native libraries
## What changes were proposed in this pull request?

Fixed the `spark-<version>-yarn-shuffle.jar` artifact packaging to shade the native netty libraries:
- shade the `META-INF/native/libnetty_*` native libraries when packagin
the yarn shuffle service jar. This is required as netty library loader
derives that based on shaded package name.
- updated the `org/spark_project` shade package prefix to `org/sparkproject`
(i.e. removed underscore) as the former breaks the netty native lib loading.

This was causing the yarn external shuffle service to fail
when spark.shuffle.io.mode=EPOLL

## How was this patch tested?
Manual tests

Closes #24502 from amuraru/SPARK-27610_master.

Authored-by: Adi Muraru <amuraru@adobe.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 10:47:36 -07:00
Wenchen Fan 6ef45301a4 [SPARK-27579][SQL] remove BaseStreamingSource and BaseStreamingSink
## What changes were proposed in this pull request?

`BaseStreamingSource` and `BaseStreamingSink` is used to unify v1 and v2 streaming data source API in some code paths.

This PR removes these 2 interfaces, and let the v1 API extend v2 API to keep API compatibility.

The motivation is https://github.com/apache/spark/pull/24416 . We want to move data source v2 to catalyst module, but `BaseStreamingSource` and `BaseStreamingSink` are in sql/core.

## How was this patch tested?

existing tests

Closes #24471 from cloud-fan/streaming.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-06 20:41:57 +08:00
Liang-Chi Hsieh 4b725e50a7 [SPARK-27439][SQL] Explainging Dataset should show correct resolved plans
## What changes were proposed in this pull request?

Because a review is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.

```scala
scala> spark.range(10).createOrReplaceTempView("test")
scala> spark.range(5).createOrReplaceTempView("test2")
scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
scala> val df = spark.sql("select * from tmp001")
scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
scala> df.show
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
scala> df.explain(true)
```

Before:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#2L]
+- SubqueryAlias `tmp001`
   +- Project [id#2L]
      +- SubqueryAlias `test2`
         +- Range (0, 5, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 5, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 5, step=1, splits=12)
```

After:
```scala
== Parsed Logical Plan ==
'Project [*]
+- 'UnresolvedRelation `tmp001`

== Analyzed Logical Plan ==
id: bigint
Project [id#0L]
+- SubqueryAlias `tmp001`
   +- Project [id#0L]
      +- SubqueryAlias `test`
         +- Range (0, 10, step=1, splits=Some(12))

== Optimized Logical Plan ==
Range (0, 10, step=1, splits=Some(12))

== Physical Plan ==
*(1) Range (0, 10, step=1, splits=12)
```

To fix it, this passes query execution of Dataset when explaining it. The query execution contains pre-analyzed plan which is consistent with Dataset's result.

## How was this patch tested?

Manually test and unit test.

Closes #24464 from viirya/SPARK-27439-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-05 23:19:19 -07:00
Dilip Biswal 6001d476ce [SPARK-27596][SQL] The JDBC 'query' option doesn't work for Oracle database
## What changes were proposed in this pull request?
**Description from JIRA**
For the JDBC option `query`, we use the identifier name to start with underscore: s"(${subquery}) _SPARK_GEN_JDBC_SUBQUERY_NAME${curId.getAndIncrement()}". This is not supported by Oracle.
The Oracle doesn't seem to support identifier name to start with non-alphabet character (unless it is quoted) and has length restrictions as well. [link](https://docs.oracle.com/cd/B19306_01/server.102/b14200/sql_elements008.htm)

In this PR, the generated alias name 'SPARK_GEN_JDBC_SUBQUERY_NAME<int value>' is fixed to remove "_" prefix and also the alias name is shortened to not exceed the identifier length limit.

## How was this patch tested?
Tests are added for MySql, Postgress, Oracle and DB2 to ensure enough coverage.

Closes #24532 from dilipbiswal/SPARK-27596.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-05 21:52:23 -07:00
Liang-Chi Hsieh d9bcacf94b [SPARK-27629][PYSPARK] Prevent Unpickler from intervening each unpickling
## What changes were proposed in this pull request?

In SPARK-27612, one correctness issue was reported. When protocol 4 is used to pickle Python objects, we found that unpickled objects were wrong. A temporary fix was proposed by not using highest protocol.

It was found that Opcodes.MEMOIZE was appeared in the opcodes in protocol 4. It is suspect to this issue.

A deeper dive found that Opcodes.MEMOIZE stores objects into internal map of Unpickler object. We use single Unpickler object to unpickle serialized Python bytes. Stored objects intervenes next round of unpickling, if the map is not cleared.

We has two options:

1. Continues to reuse Unpickler, but calls its close after each unpickling.
2. Not to reuse Unpickler and create new Unpickler object in each unpickling.

This patch takes option 1.

## How was this patch tested?

Passing the test added in SPARK-27612 (#24519).

Closes #24521 from viirya/SPARK-27629.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 13:21:08 +09:00
Seth Fitzsimmons 5182aa25f0 [MINOR][DOCS] Correct date_trunc docs
## What changes were proposed in this pull request?

`date_trunc` argument order was flipped, phrasing was awkward.

## How was this patch tested?

Documentation-only.

Closes #24522 from mojodna/patch-2.

Authored-by: Seth Fitzsimmons <seth@mojodna.net>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 09:13:23 +09:00
sandeep katta c66ec43945 [SPARK-27555][SQL] HiveSerDe should fall back to hadoopconf if hive.default.fileformat is not found in SQLConf
## What changes were proposed in this pull request?

SQLConf does not load hive-site.xml.So HiveSerDe should fall back to hadoopconf if  hive.default.fileformat is not found in SQLConf

## How was this patch tested?

Tested manually.
Added UT

Closes #24489 from sandeep-katta/spark-27555.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 09:02:12 +09:00
gaoweikang 3859ca37d9 [SPARK-27586][SQL] Improve binary comparison: replace Scala's for-comprehension if statements with while loop
## What changes were proposed in this pull request?

This PR replaces for-comprehension if statement with while loop to gain better performance in `TypeUtils.compareBinary`.

## How was this patch tested?

Add UT to test old version and new version comparison result

Closes #24494 from woudygao/opt_binary_compare.

Authored-by: gaoweikang <gaoweikang@bytedance.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-02 20:33:27 -07:00
Marco Gaido 7a8cc8e071 [SPARK-27607][SQL] Improve Row.toString performance
## What changes were proposed in this pull request?

`Row.toString` is currently causing the useless creation of an `Array` containing all the values in the row before generating the string containing it. This operation adds a considerable overhead.

The PR proposes to avoid this operation in order to get a faster implementation.

## How was this patch tested?

Run

```scala
test("Row toString perf test") {
    val n = 100000
    val rows = (1 to n).map { i =>
      Row(i, i.toDouble, i.toString, i.toShort, true, null)
    }
    // warmup
    (1 to 10).foreach { _ => rows.foreach(_.toString) }

    val times = (1 to 100).map { _ =>
      val t0 = System.nanoTime()
      rows.foreach(_.toString)
      val t1 = System.nanoTime()
      t1 - t0
    }
    // scalastyle:off println
    println(s"Avg time on ${times.length} iterations for $n toString:" +
      s" ${times.sum.toDouble / times.length / 1e6} ms")
    // scalastyle:on println
  }
```
Before the PR:
```
Avg time on 100 iterations for 100000 toString: 61.08408419 ms
```
After the PR:
```
Avg time on 100 iterations for 100000 toString: 38.16539432 ms
```
This means the new implementation is about 1.60X faster than the original one.

Closes #24505 from mgaido91/SPARK-27607.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-02 07:20:33 -07:00
HyukjinKwon df8aa7ba8a [SPARK-27606][SQL] Deprecate 'extended' field in ExpressionDescription/ExpressionInfo
## What changes were proposed in this pull request?

After we added other fields, `arguments`, `examples`, `note` and `since` at SPARK-21485 and `deprecated` at SPARK-27328, we have nicer way to separately describe extended usages.

`extended` field and method at `ExpressionDescription`/`ExpressionInfo` is now pretty useless - it's not used in Spark side and only exists to keep backward compatibility.

This PR proposes to deprecate it.

## How was this patch tested?

Manually checked the deprecation waring is properly shown.

Closes #24500 from HyukjinKwon/SPARK-27606.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-02 21:10:00 +09:00
gatorsmile 2da406cae5 [SPARK-27618][SQL][FOLLOW-UP] Unnecessary access to externalCatalog
## What changes were proposed in this pull request?
This PR is to add test cases for ensuring that we do not have unnecessary access to externalCatalog.

In the future, we can follow these examples to improve our test coverage in this area.

## How was this patch tested?
N/A

Closes #24511 from gatorsmile/addTestcaseSpark-27618.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-01 20:09:46 -07:00
HyukjinKwon 3670826af6 [SPARK-26921][R][DOCS] Document Arrow optimization and vectorized R APIs
## What changes were proposed in this pull request?

This PR adds SparkR with Arrow optimization documentation.

Note that looks CRAN issue in Arrow side won't look likely fixed soon, IMHO, even after Spark 3.0.
If it happen to be fixed, I will fix this doc too later.

Another note is that Arrow R package itself requires R 3.5+. So, I intentionally didn't note this.

## How was this patch tested?

Manually built and checked.

Closes #24506 from HyukjinKwon/SPARK-26924.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-02 10:02:14 +09:00
Artem Kalchenko a35043c9e2 [SPARK-27591][SQL] Fix UnivocityParser for UserDefinedType
## What changes were proposed in this pull request?

Fix bug in UnivocityParser. makeConverter method didn't work correctly for UsedDefinedType

## How was this patch tested?

A test suite for UnivocityParser has been extended.

Closes #24496 from kalkolab/spark-27591.

Authored-by: Artem Kalchenko <artem.kalchenko@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-01 08:27:51 +09:00
Sean Owen 25ee0474f4 [SPARK-26936][MINOR][FOLLOWUP] Don't need the JobConf anymore, it seems
## What changes were proposed in this pull request?

On a second look in comments, seems like the JobConf isn't needed anymore here. It was used inconsistently before, it seems, and I don't see any reason a Hadoop Job config is required here anyway.

## How was this patch tested?

Existing tests.

Closes #24491 from srowen/SPARK-26936.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-29 19:47:20 -07:00
Wenchen Fan 7432e7ded4 [SPARK-24935][SQL][FOLLOWUP] support INIT -> UPDATE -> MERGE -> FINISH in Hive UDAF adapter
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24144 . #24144 missed one case: when hash aggregate fallback to sort aggregate, the life cycle of UDAF is: INIT -> UPDATE -> MERGE -> FINISH.

However, not all Hive UDAF can support it. Hive UDAF knows the aggregation mode when creating the aggregation buffer, so that it can create different buffers for different inputs: the original data or the aggregation buffer. Please see an example in the [sketches library](7f9e76e9e0/src/main/java/com/yahoo/sketches/hive/cpc/DataToSketchUDAF.java (L107)). The buffer for UPDATE may not support MERGE.

This PR updates the Hive UDAF adapter in Spark to support INIT -> UPDATE -> MERGE -> FINISH, by turning it to  INIT -> UPDATE -> FINISH + IINIT -> MERGE -> FINISH.

## How was this patch tested?

a new test case

Closes #24459 from cloud-fan/hive-udaf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-30 10:35:23 +08:00
Xiangrui Meng 618d6bff71 [SPARK-27588] Binary file data source fails fast and doesn't attempt to read very large files
## What changes were proposed in this pull request?

If a file is too big (>2GB), we should fail fast and do not try to read the file.

## 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 http://spark.apache.org/contributing.html before opening a pull request.

Closes #24483 from mengxr/SPARK-27588.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-29 16:24:49 -07:00
Gabor Somogyi fb6b19ab7c [SPARK-23014][SS] Fully remove V1 memory sink.
## What changes were proposed in this pull request?

There is a MemorySink v2 already so v1 can be removed. In this PR I've removed it completely.
What this PR contains:
* V1 memory sink removal
* V2 memory sink renamed to become the only implementation
* Since DSv2 sends exceptions in a chained format (linking them with cause field) I've made python side compliant
* Adapted all the tests

## How was this patch tested?

Existing unit tests.

Closes #24403 from gaborgsomogyi/SPARK-23014.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 09:44:23 -07:00
Sean Owen a6716d3f03 [SPARK-27571][CORE][YARN][EXAMPLES] Avoid scala.language.reflectiveCalls
## What changes were proposed in this pull request?

This PR avoids usage of reflective calls in Scala. It removes the import that suppresses the warnings and rewrites code in small ways to avoid accessing methods that aren't technically accessible.

## How was this patch tested?

Existing tests.

Closes #24463 from srowen/SPARK-27571.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:16:45 -05:00
Sean Owen 8a17d26784 [SPARK-27536][CORE][ML][SQL][STREAMING] Remove most use of scala.language.existentials
## What changes were proposed in this pull request?

I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785

For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.

This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.

Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.

Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.

For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.

One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.

In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.

Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.

After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.

## How was this patch tested?

Existing tests.

Closes #24431 from srowen/SPARK-27536.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:02:01 -05:00
Liang-Chi Hsieh 76785cd6f0 [SPARK-27581][SQL] DataFrame countDistinct("*") shouldn't fail with AnalysisException
## What changes were proposed in this pull request?

Currently `countDistinct("*")` doesn't work. An analysis exception is thrown:

```scala
val df = sql("select id % 100 from range(100000)")
df.select(countDistinct("*")).first()

org.apache.spark.sql.AnalysisException: Invalid usage of '*' in expression 'count';
```

Users need to use `expr`.

```scala
df.select(expr("count(distinct(*))")).first()
```

This limits some API usage like `df.select(count("*"), countDistinct("*))`.

The PR takes the simplest fix that lets analyzer expand star and resolve `count` function.

## How was this patch tested?

Added unit test.

Closes #24482 from viirya/SPARK-27581.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 21:17:32 +08:00
Yuming Wang 5a62295219 [SPARK-27580][HOT-FIX] Fix wrong import order in FileScan.scala
## What changes were proposed in this pull request?
```
========================================================================
Running Scala style checks
========================================================================
[info] Checking Scala style using SBT with these profiles:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos
Scalastyle checks failed at following occurrences:
[error] /home/jenkins/workspace/SparkPullRequestBuilder/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala:29:0: org.apache.spark.sql.sources.Filter is in wrong order relative to org.apache.spark.sql.sources.v2.reader..
[error] Total time: 17 s, completed Apr 29, 2019 3:09:43 AM
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/104987/console

## How was this patch tested?
manual tests:
```
dev/scalastyle
```

Closes #24487 from wangyum/SPARK-27580.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 20:48:12 +08:00
Gengliang Wang 07d07fec03 [SPARK-27580][SQL] Implement doCanonicalize in BatchScanExec for comparing query plan results
## What changes were proposed in this pull request?

The method `QueryPlan.sameResult` is used for comparing logical plans in order to:
1. cache data in CacheManager
2. uncache data in CacheManager
3. Reuse subqueries
4. etc...

Currently the method `sameReuslt` always return false for `BatchScanExec`. We should fix it by implementing `doCanonicalize` for the node.

## How was this patch tested?

Unit test

Closes #24475 from gengliangwang/sameResultForV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 17:54:12 +08:00
Xiangrui Meng 20a3ef7259 [SPARK-27534][SQL] Do not load content column in binary data source if it is not selected
## What changes were proposed in this pull request?

A follow-up task from SPARK-25348. To save I/O cost, Spark shouldn't attempt to read the file if users didn't request the `content` column. For example:
```
spark.read.format("binaryFile").load(path).filter($"length" < 1000000).count()
```

## How was this patch tested?

Unit test added.

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

Closes #24473 from WeichenXu123/SPARK-27534.

Lead-authored-by: Xiangrui Meng <meng@databricks.com>
Co-authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-28 07:57:03 -07:00
Jash Gala 90085a1847 [SPARK-23619][DOCS] Add output description for some generator expressions / functions
## What changes were proposed in this pull request?

This PR addresses SPARK-23619: https://issues.apache.org/jira/browse/SPARK-23619

It adds additional comments indicating the default column names for the `explode` and `posexplode`
functions in Spark-SQL.

Functions for which comments have been updated so far:
* stack
* inline
* explode
* posexplode
* explode_outer
* posexplode_outer

## How was this patch tested?

This is just a change in the comments. The package builds and tests successfullly after the change.

Closes #23748 from jashgala/SPARK-23619.

Authored-by: Jash Gala <jashgala@amazon.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-27 10:30:12 +09:00
uncleGen 6328be78f9 [MINOR][TEST][DOC] Execute action miss name message
## What changes were proposed in this pull request?

some minor updates:
- `Execute` action miss `name` message
-  typo in SS document
-  typo in SQLConf

## How was this patch tested?

N/A

Closes #24466 from uncleGen/minor-fix.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-27 09:28:31 +08:00
Wenchen Fan 85fd552ed6 [SPARK-27190][SQL] add table capability for streaming
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24012 , to add the corresponding capabilities for streaming.

## How was this patch tested?

existing tests

Closes #24129 from cloud-fan/capability.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-26 15:44:23 +08:00
Wenchen Fan 2234667b15 [SPARK-27563][SQL][TEST] automatically get the latest Spark versions in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

We can get the latest downloadable Spark versions from https://dist.apache.org/repos/dist/release/spark/

## How was this patch tested?

manually.

Closes #24454 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-26 16:37:43 +09:00
Dongjoon Hyun d5dbf053d3 Revert "[SPARK-27439][SQL] Use analyzed plan when explaining Dataset"
This reverts commit ad60c6d9be.
2019-04-25 18:38:52 -07:00
Liang-Chi Hsieh 8b86326521 [SPARK-27551][SQL] Improve error message of mismatched types for CASE WHEN
## What changes were proposed in this pull request?

When there are mismatched types among cases or else values in case when expression, current error message is hard to read to figure out what and where the mismatch is.

This patch simply improves the error message for mismatched types for case when.

Before:
```scala
scala> spark.range(100).select(when('id === 1, array(struct('id * 123456789 + 123456789 as "x"))).otherwise(array(struct('id * 987654321 + 987654321 as
 "y"))))
org.apache.spark.sql.AnalysisException: cannot resolve 'CASE WHEN (`id` = CAST(1 AS BIGINT)) THEN array(named_struct('x', ((`id` * CAST(123456789 AS BI
GINT)) + CAST(123456789 AS BIGINT)))) ELSE array(named_struct('y', ((`id` * CAST(987654321 AS BIGINT)) + CAST(987654321 AS BIGINT)))) END' due to data
type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type;;
```

After:
```scala
scala> spark.range(100).select(when('id === 1, array(struct('id * 123456789 + 123456789 as "x"))).otherwise(array(struct('id * 987654321 + 987654321 as
 "y"))))
org.apache.spark.sql.AnalysisException: cannot resolve 'CASE WHEN (`id` = CAST(1 AS BIGINT)) THEN array(named_struct('x', ((`id` * CAST(123456789 AS BI
GINT)) + CAST(123456789 AS BIGINT)))) ELSE array(named_struct('y', ((`id` * CAST(987654321 AS BIGINT)) + CAST(987654321 AS BIGINT)))) END' due to data
type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN array<struct<x:bigint>> ELSE arr
ay<struct<y:bigint>> END;;
```

## How was this patch tested?

Added unit test.

Closes #24453 from viirya/SPARK-27551.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-25 08:47:19 -07:00
Wenchen Fan b7f9830670 [MINOR][TEST] switch from 2.4.1 to 2.4.2 in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

update `HiveExternalCatalogVersionsSuite` to test 2.4.2, as 2.4.1 will be removed from Mirror Network soon.

## How was this patch tested?

N/A

Closes #24452 from cloud-fan/release.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-25 10:26:40 +08:00
gatorsmile cd4a284030 [SPARK-27460][FOLLOW-UP][TESTS] Fix flaky tests
## What changes were proposed in this pull request?

This patch makes several test flakiness fixes.

## How was this patch tested?
N/A

Closes #24434 from gatorsmile/fixFlakyTest.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-24 17:36:29 +08:00
HyukjinKwon a30983db57 [SPARK-27512][SQL] Avoid to replace ',' in CSV's decimal type inference for backward compatibility
## What changes were proposed in this pull request?

The code below currently infers as decimal but previously it was inferred as string.

**In branch-2.4**, type inference path for decimal and parsing data are different.

2a8343121e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala (L153)

c284c4e1f6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala (L125)

So the code below:

```scala
scala> spark.read.option("delimiter", "|").option("inferSchema", "true").csv(Seq("1,2").toDS).printSchema()
```

produced string as its type.

```
root
 |-- _c0: string (nullable = true)
```

**In the current master**, it now infers decimal as below:

```
root
 |-- _c0: decimal(2,0) (nullable = true)
```

It happened after https://github.com/apache/spark/pull/22979 because, now after this PR, we only have one way to parse decimal:

7a83d71403/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala (L92)

**After the fix:**

```
root
 |-- _c0: string (nullable = true)
```

This PR proposes to restore the previous behaviour back in `CSVInferSchema`.

## How was this patch tested?

Manually tested and unit tests were added.

Closes #24437 from HyukjinKwon/SPARK-27512.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 16:22:07 +09:00
Sean Owen 596a5ff273 [MINOR][BUILD] Update genjavadoc to 0.13
## What changes were proposed in this pull request?

Kind of related to https://github.com/gatorsmile/spark/pull/5 - let's update genjavadoc to see if it generates fewer spurious javadoc errors to begin with.

## How was this patch tested?

Existing docs build

Closes #24443 from srowen/genjavadoc013.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 13:44:48 +09:00
Gengliang Wang 00f2f311f7 [SPARK-27128][SQL] Migrate JSON to File Data Source V2
## What changes were proposed in this pull request?
Migrate JSON to File Data Source V2

## How was this patch tested?

Unit test

Closes #24058 from gengliangwang/jsonV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-23 22:39:59 +08:00
pengbo d9b2ce0f0f [SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values
## What changes were proposed in this pull request?
This PR is follow up of https://github.com/apache/spark/pull/24286. As gatorsmile pointed out that column with null value is inaccurate as well.

```
> select key from test;
2
NULL
1
spark-sql> desc extended test key;
col_name key
data_type int
comment NULL
min 1
max 2
num_nulls 1
distinct_count 2
```

The distinct count should be distinct_count + 1 when column contains null value.
## How was this patch tested?

Existing tests & new UT added.

Closes #24436 from pengbo/aggregation_estimation.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-22 20:30:08 -07:00
Maxim Gekk 93a264d05a [SPARK-27535][SQL][TEST] Date and timestamp JSON benchmarks
## What changes were proposed in this pull request?

Added new JSON benchmarks related to date and timestamps operations:
- Write date/timestamp to JSON files
- `to_json()` and `from_json()` for dates and timestamps
- Read date/timestamps from JSON files, and infer schemas
- Parse and infer schemas from `Dataset[String]`

Also existing JSON benchmarks are ported on `NoOp` datasource.

Closes #24430 from MaxGekk/json-datetime-benchmark.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:09:14 +09:00
Maxim Gekk 55f26d8090 [SPARK-27533][SQL][TEST] Date and timestamp CSV benchmarks
## What changes were proposed in this pull request?

Added new CSV benchmarks related to date and timestamps operations:
- Write date/timestamp to CSV files
- `to_csv()` and `from_csv()` for dates and timestamps
- Read date/timestamps from CSV files, and infer schemas
- Parse and infer schemas from `Dataset[String]`

Also existing CSV benchmarks are ported on `NoOp` datasource.

Closes #24429 from MaxGekk/csv-timestamp-benchmark.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:08:02 +09:00
Maxim Gekk 43a73e387c [SPARK-27528][SQL] Use Parquet logical type TIMESTAMP_MICROS by default
## What changes were proposed in this pull request?

In the PR, I propose to use the `TIMESTAMP_MICROS` logical type for timestamps written to parquet files. The type matches semantically to Catalyst's `TimestampType`, and stores microseconds since epoch in UTC time zone. This will allow to avoid conversions of microseconds to nanoseconds and to Julian calendar. Also this will reduce sizes of written parquet files.

## How was this patch tested?

By existing test suites.

Closes #24425 from MaxGekk/parquet-timestamp_micros.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:06:39 +09:00
Dilip Biswal 3240e52dc7 [SPARK-27531][SQL] Improve EXPLAIN DESC TABLE to show the input parameters of the command.
## What changes were proposed in this pull request?
Currently "EXPLAIN DESC TABLE" is special cased and outputs a single row relation as following.
Current output:
```sql
spark-sql> EXPLAIN DESCRIBE TABLE t;
== Physical Plan ==
*(1) Scan OneRowRelation[]
```
This is not consistent with how we handle explain processing for other commands. In this PR, the inconsistency is handled by removing the special handling for "describe table".

After change:
```sql
spark-sql> EXPLAIN DESC EXTENDED t
== Physical Plan ==
Execute DescribeTableCommand
   +- DescribeTableCommand `t`, true
```
## How was this patch tested?
Added new tests in SQLQueryTestSuite.

Closes #24427 from dilipbiswal/describe_table_explain2.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-22 13:02:10 -07:00
Eric Liang 5172190da1 [SPARK-27392][SQL] TestHive test tables should be placed in shared test state, not per session
## What changes were proposed in this pull request?

Otherwise, tests that use tables from multiple sessions will run into issues if they access the same table. The correct location is in shared state.

A couple other minor test improvements.

cc gatorsmile srinathshankar

## How was this patch tested?

Existing unit tests.

Closes #24302 from ericl/test-conflicts.

Lead-authored-by: Eric Liang <ekl@databricks.com>
Co-authored-by: Eric Liang <ekhliang@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-22 11:05:31 -07:00
Maxim Gekk 79d3bc0409 [SPARK-27438][SQL] Parse strings with timestamps by to_timestamp() in microsecond precision
## What changes were proposed in this pull request?

In the PR, I propose to parse strings to timestamps in microsecond precision by the ` to_timestamp()` function if the specified pattern contains a sub-pattern for seconds fractions.

Closes #24342

## How was this patch tested?

By `DateFunctionsSuite` and `DateExpressionsSuite`

Closes #24420 from MaxGekk/to_timestamp-microseconds3.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-22 19:41:32 +08:00
Maxim Gekk 777b797867 [SPARK-27522][SQL][TEST] Test migration from INT96 to TIMESTAMP_MICROS for timestamps in parquet
## What changes were proposed in this pull request?

Added tests to check migration from `INT96` to `TIMESTAMP_MICROS` (`INT64`) for timestamps in parquet files. In particular:
- Append `TIMESTAMP_MICROS` timestamps to **existing parquet** files with `INT96` timestamps
- Append `TIMESTAMP_MICROS` timestamps to a table with `INT96` timestamps
- Append `INT96` to `TIMESTAMP_MICROS` timestamps in **parquet files**
- Append `INT96` to `TIMESTAMP_MICROS` timestamps in a **table**

Closes #24417 from MaxGekk/parquet-timestamp-int64-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-22 16:34:13 +09:00
Dilip Biswal 8a8643c28d [SPARK-27480][SQL] Improve EXPLAIN DESC QUERY to show the input SQL statement
Currently running explain on describe query gives a little confusing output. This is a minor pr that improves the output of explain.

Before
```
1.EXPLAIN DESCRIBE WITH s AS (SELECT 'hello' as col1) SELECT * FROM s;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand CTE [s]
2.EXPLAIN EXTENDED DESCRIBE SELECT * from s1 where c1 > 0;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand 'Project [*]
```
After
```
1. EXPLAIN DESCRIBE WITH s AS (SELECT 'hello' as col1) SELECT * FROM s;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s
2. EXPLAIN DESCRIBE SELECT * from s1 where c1 > 0;
== Physical Plan ==
Execute DescribeQueryCommand
   +- DescribeQueryCommand SELECT * from s1 where c1 > 0
```
Added a couple of tests in describe-query.sql under SQLQueryTestSuite.

Closes #24385 from dilipbiswal/describe_query_explain.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-21 15:35:05 -07:00
WeichenXu 9793d9ec22 [SPARK-27473][SQL] Support filter push down for status fields in binary file data source
## What changes were proposed in this pull request?

Support 4 kinds of filters:
- LessThan
- LessThanOrEqual
- GreatThan
- GreatThanOrEqual

Support filters applied on 2 columns:
- modificationTime
- length

Note:
In order to support datasource filter push-down, I flatten schema to be:
```
val schema = StructType(
    StructField("path", StringType, false) ::
    StructField("modificationTime", TimestampType, false) ::
    StructField("length", LongType, false) ::
    StructField("content", BinaryType, true) :: Nil)
```

## How was this patch tested?

To be added.

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

Closes #24387 from WeichenXu123/binary_ds_filter.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-21 12:45:59 -07:00
Liang-Chi Hsieh ad60c6d9be [SPARK-27439][SQL] Use analyzed plan when explaining Dataset
## What changes were proposed in this pull request?

Because a review is resolved during analysis when we create a dataset, the content of the view is determined when the dataset is created, not when it is evaluated. Now the explain result of a dataset is not correctly consistent with the collected result of it, because we use pre-analyzed logical plan of the dataset in explain command. The explain command will analyzed the logical plan passed in. So if a view is changed after the dataset was created, the plans shown by explain command aren't the same with the plan of the dataset.

```scala
scala> spark.range(10).createOrReplaceTempView("test")
scala> spark.range(5).createOrReplaceTempView("test2")
scala> spark.sql("select * from test").createOrReplaceTempView("tmp001")
scala> val df = spark.sql("select * from tmp001")
scala> spark.sql("select * from test2").createOrReplaceTempView("tmp001")
scala> df.show
+---+
| id|
+---+
|  0|
|  1|
|  2|
|  3|
|  4|
|  5|
|  6|
|  7|
|  8|
|  9|
+---+
scala> df.explain
```

Before:
```scala
== Physical Plan ==
*(1) Range (0, 5, step=1, splits=12)
```

After:
```scala
== Physical Plan ==
*(1) Range (0, 10, step=1, splits=12)

```

## How was this patch tested?

Manually test and unit test.

Closes #24415 from viirya/SPARK-27439.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-21 10:25:56 -07:00
Maxim Gekk d61b3bc875 [SPARK-27527][SQL][DOCS] Improve descriptions of Timestamp and Date types
## What changes were proposed in this pull request?

In the PR, I propose more precise description of `TimestampType` and `DateType`, how they store timestamps and dates internally.

Closes #24424 from MaxGekk/timestamp-date-type-doc.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-21 16:53:11 +09:00
Gengliang Wang 31488e1ca5 [SPARK-27504][SQL] File source V2: support refreshing metadata cache
## What changes were proposed in this pull request?

In file source V1, if some file is deleted manually, reading the DataFrame/Table will throws an exception with suggestion message
```
It is possible the underlying files have been updated. You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved.
```
After refreshing the table/DataFrame, the reads should return correct results.

We should follow it in file source V2 as well.
## How was this patch tested?
Unit test

Closes #24401 from gengliangwang/refreshFileTable.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-19 18:26:03 +08:00
Yifei Huang 163a6e2982 [SPARK-27514] Skip collapsing windows with empty window expressions
## What changes were proposed in this pull request?

A previous change moved the removal of empty window expressions to the RemoveNoopOperations rule, which comes after the CollapseWindow rule. Therefore, by the time we get to CollapseWindow, we aren't guaranteed that empty windows have been removed. This change checks that the window expressions are not empty, and only collapses the windows if both windows are non-empty.

A lengthier description and repro steps here: https://issues.apache.org/jira/browse/SPARK-27514

## How was this patch tested?

A unit test, plus I reran the breaking case mentioned in the Jira ticket.

Closes #24411 from yifeih/yh/spark-27514.

Authored-by: Yifei Huang <yifeih@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-19 14:04:44 +08:00
Yuming Wang 8f82237a5b [SPARK-27501][SQL][TEST] Add test for HIVE-13083: Writing HiveDecimal to ORC can wrongly suppress present stream
## What changes were proposed in this pull request?

This PR add test for [HIVE-13083](https://issues.apache.org/jira/browse/HIVE-13083): Writing HiveDecimal to ORC can wrongly suppress present stream.

## How was this patch tested?
manual tests:
```
build/sbt  "hive/testOnly *HiveOrcQuerySuite" -Phive -Phadoop-3.2
```

Closes #24397 from wangyum/SPARK-26437.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-19 10:12:21 +09:00
Gengliang Wang 3748b381df [SPARK-27460][TESTS][FOLLOWUP] Add HiveClientVersions to parallel test suite list
## What changes were proposed in this pull request?

The test time of `HiveClientVersions` is around 3.5 minutes.
This PR is to add it into the parallel test suite list. To make sure there is no colliding warehouse location,  we can change the warehouse path to a temporary directory.

## How was this patch tested?

Unit test

Closes #24404 from gengliangwang/parallelTestFollowUp.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-18 15:37:55 -07:00
Liang-Chi Hsieh 9c41bfd83c [SPARK-27502][SQL][TEST] Update nested schema benchmark result for Orc V2
## What changes were proposed in this pull request?

We added nested schema pruning support to Orc V2 recently. The benchmark result should be updated. The benchmark numbers are obtained by running benchmark on r3.xlarge machine.

## How was this patch tested?

Test only change.

Closes #24399 from viirya/update-orcv2-benchmark.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-18 08:08:22 -07:00
Gengliang Wang 9c238b8a46 [SPARK-27460][TESTS] Running slowest test suites in their own forked JVMs for higher parallelism
## What changes were proposed in this pull request?

This patch modifies SparkBuild so that the largest / slowest test suites (or collections of suites) can run in their own forked JVMs, allowing them to be run in parallel with each other. This opt-in / whitelisting approach allows us to increase parallelism without having to fix a long-tail of flakiness / brittleness issues in tests which aren't performance bottlenecks.

See comments in SparkBuild.scala for information on the details, including a summary of why we sometimes opt to run entire groups of tests in a single forked JVM .

The time of full new pull request test in Jenkins is reduced by around 53%:
before changes: 4hr 40min
after changes: 2hr 13min

## How was this patch tested?

Unit test

Closes #24373 from gengliangwang/parallelTest.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 20:49:36 +08:00
Gengliang Wang 7d44ba05d1 [SPARK-27490][SQL] File source V2: return correct result for Dataset.inputFiles()
## What changes were proposed in this pull request?

Currently, a `Dateset` with file source V2 always return empty results for method `Dataset.inputFiles()`.

We should fix it.

## How was this patch tested?

Unit test

Closes #24393 from gengliangwang/inputFiles.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 14:39:30 +08:00
Kris Mok 50bdc9befa [SPARK-27423][SQL][FOLLOWUP] Minor polishes to Cast codegen templates for Date <-> Timestamp
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/24332 introduced an unnecessary `import` statement and two slight issues in the codegen templates in `Cast` for `Date` <-> `Timestamp`.
This PR removes the unused import statement and fixes the slight codegen issue.

The issue in those two codegen templates is this pattern:
```scala
val zid = JavaCode.global(
  ctx.addReferenceObj("zoneId", zoneId, "java.time.ZoneId"),
  zoneId.getClass)
```
`zoneId` can refer to an instance of a non-public class, e.g. `java.time.ZoneRegion`, and while this code correctly puts in the 3rd argument to `ctx.addReferenceObj()`, it's still passing `zoneId.getClass` to `JavaCode.global()` which is not desirable, but doesn't cause any immediate bugs in this particular case, because `zid` is used in an expression immediately afterwards.
If this `zid` ever needs to spill to any explicitly typed variables, e.g. a local variable, and if the spill handling uses the `javaType` on this `GlobalVariable`, it'd generate code that looks like:
```java
java.time.ZoneRegion value1 = ((java.time.ZoneId) references[2] /* literal */);
```
which would then be a real bug:
- a non-accessible type `java.time.ZoneRegion` is referenced in the generated code, and
- `ZoneId` -> `ZoneRegion` requires an explicit downcast.

## How was this patch tested?

Existing tests. This PR does not change behavior, and the original PR won't cause any real behavior bug to begin with.

Closes #24392 from rednaxelafx/spark-27423-followup.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 14:27:33 +08:00
Dilip Biswal e1c90d66bb [SPARK-19712][SQL] Pushdown LeftSemi/LeftAnti below join
## What changes were proposed in this pull request?
This PR adds support for pushing down LeftSemi and LeftAnti joins below the Join operator.
This is a prerequisite work thats needed for the subsequent task of moving the subquery rewrites to the beginning of optimization phase.

The larger  PR is [here](https://github.com/apache/spark/pull/23211) . This PR addresses the comment at [link](https://github.com/apache/spark/pull/23211#issuecomment-445705922).
## How was this patch tested?
Added tests under LeftSemiAntiJoinPushDownSuite.

Closes #24331 from dilipbiswal/SPARK-19712-pushleftsemi-belowjoin.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 20:30:20 +08:00
Wenchen Fan e6618de809 [SPARK-27430][SQL] broadcast hint should be respected for broadcast nested loop join
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/24164

broadcast hint should be respected for broadcast nested loop join. This PR also refactors the related code a little bit, to save duplicated code.

## How was this patch tested?

new tests

Closes #24376 from cloud-fan/join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 19:29:28 +08:00
pengbo 54b0d1e0ef [SPARK-27416][SQL] UnsafeMapData & UnsafeArrayData Kryo serialization …
## What changes were proposed in this pull request?
Finish the rest work of https://github.com/apache/spark/pull/24317, https://github.com/apache/spark/pull/9030
a. Implement Kryo serialization for UnsafeArrayData
b. fix UnsafeMapData Java/Kryo Serialization issue when two machines have different Oops size
c. Move the duplicate code "getBytes()" to Utils.

## How was this patch tested?
According Units has been added & tested

Closes #24357 from pengbo/SPARK-27416_new.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 13:03:00 +08:00
WeichenXu 1bb0c8e407 [SPARK-25348][SQL] Data source for binary files
## What changes were proposed in this pull request?

Implement binary file data source in Spark.

Format name: "binaryFile" (case-insensitive)

Schema:
- content: BinaryType
- status: StructType
  - path: StringType
  - modificationTime: TimestampType
  - length: LongType

Options:
* pathGlobFilter (instead of pathFilterRegex) to reply on GlobFilter behavior
* maxBytesPerPartition is not implemented since it is controlled by two SQL confs: maxPartitionBytes and openCostInBytes.

## How was this patch tested?

Unit test added.

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

Closes #24354 from WeichenXu123/binary_file_datasource.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-16 15:41:32 -07:00
liwensun 26ed65f415 [SPARK-27453] Pass partitionBy as options in DataFrameWriter
## What changes were proposed in this pull request?

Pass partitionBy columns as options and feature-flag this behavior.

## How was this patch tested?

A new unit test.

Closes #24365 from liwensun/partitionby.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2019-04-16 15:03:16 -07:00
Liang-Chi Hsieh b404e02574 [SPARK-27476][SQL] Refactoring SchemaPruning rule to remove duplicate code
## What changes were proposed in this pull request?

In SchemaPruning rule, there is duplicate code for data source v1 and v2. Their logic is the same and we can refactor the rule to remove duplicate code.

## How was this patch tested?

Existing tests.

Closes #24383 from viirya/SPARK-27476.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-16 14:50:37 -07:00
shivusondur 88d9de26dd [SPARK-27464][CORE] Added Constant instead of referring string literal used from many places
## What changes were proposed in this pull request?

Added Constant instead of referring the same String literal "spark.buffer.pageSize" from many places
## How was this patch tested?
Run the corresponding Unit Test Cases manually.

Closes #24368 from shivusondur/Constant.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-16 09:30:46 -05:00
Gengliang Wang f9837d3bf6 [SPARK-27448][SQL] File source V2 table provider should be compatible with V1 provider
## What changes were proposed in this pull request?

In the rule `PreprocessTableCreation`, if an existing table is appended with a different provider, the action will fail.
Currently, there are two implementations for file sources and creating a table with file source V2 will always fall back to V1 FileFormat. We should consider the following cases as valid:
1. Appending a table with file source V2 provider using the v1 file format
2. Appending a table with v1 file format provider using file source V2 format

## How was this patch tested?

Unit test

Closes #24356 from gengliangwang/fixTableProvider.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-16 14:26:38 +08:00
pengbo c58a4fed8d [SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…
## What changes were proposed in this pull request?
The upper bound of group-by columns row number is to multiply distinct counts of group-by columns. However, column with only null value will cause the output row number to be 0 which is incorrect.
Ex:
col1 (distinct: 2, rowCount 2)
col2 (distinct: 0, rowCount 2)
=> group by col1, col2
Actual: output rows: 0
Expected: output rows: 2

## How was this patch tested?
According unit test has been added, plus manual test has been done in our tpcds benchmark environement.

Closes #24286 from pengbo/master.

Lead-authored-by: pengbo <bo.peng1019@gmail.com>
Co-authored-by: mingbo_pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-15 15:37:07 -07:00
Dilip Biswal 3ab96d7acf [SPARK-27444][SQL][FOLLOWUP][MINOR][TEST] Add a test for describing multi select query.
## What changes were proposed in this pull request?
This is a minor pr to add a test to describe a multi select query.

## How was this patch tested?
Added a test in describe-query.sql

Closes #24370 from dilipbiswal/describe-query-multiselect-test.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-15 21:26:45 +08:00
Gengliang Wang 27d625d785 [SPARK-27459][SQL] Revise the exception message of schema inference failure in file source V2
## What changes were proposed in this pull request?

Since https://github.com/apache/spark/pull/23383/files#diff-db4a140579c1ac4b1dbec7fe5057eecaR36, the exception message of schema inference failure in file source V2 is `tableName`, which is equivalent to `shortName + path`.

While in file source V1, the message is `Unable to infer schema from ORC/CSV/JSON...`.
We should make the message in V2 consistent with V1, so that in the future migration the related test cases don't need to be modified. https://github.com/apache/spark/pull/24058#pullrequestreview-226364350

## How was this patch tested?

Revert the modified unit test cases in https://github.com/apache/spark/pull/24005/files#diff-b9ddfbc9be8d83ecf100b3b8ff9610b9R431 and https://github.com/apache/spark/pull/23383/files#diff-9ab56940ee5a53f2bb81e3c008653362R577, and test with them.

Closes #24369 from gengliangwang/reviseInferSchemaMessage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-15 21:06:03 +08:00
herman 4704af4c26 [SPARK-27449] Move WholeStageCodegen.limitNotReachedCond class checks into separate methods.
## What changes were proposed in this pull request?
This PR moves the checks done in `WholeStageCodegen.limitNotReachedCond` into a separate protected method. This makes it easier to introduce new leaf or blocking nodes.

## How was this patch tested?
Existing tests.

Closes #24358 from hvanhovell/SPARK-27449.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-14 15:54:20 +08:00
Wenchen Fan 0407070945 [SPARK-27444][SQL] multi-select can be used in subquery
## What changes were proposed in this pull request?

This is a regression caused by https://github.com/apache/spark/pull/24150

`select * from (from a select * select *)` is supported in 2.4, and we should keep supporting it.

This PR merges the parser rule for single and multi select statements, as they are very similar.

## How was this patch tested?

a new test case

Closes #24348 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 20:57:34 +08:00
Gengliang Wang 4eb694c58f [SPARK-27443][SQL] Support UDF input_file_name in file source V2
## What changes were proposed in this pull request?

Currently, if we select the UDF `input_file_name` as a column in file source V2, the results are empty.
We should support it in file source V2.

## How was this patch tested?

Unit test

Closes #24347 from gengliangwang/input_file_name.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 20:30:42 +08:00
Kris Mok bbbe54aa79 [SPARK-27199][SQL][FOLLOWUP] Fix bug in codegen templates in UnixTime and FromUnixTime
## What changes were proposed in this pull request?

SPARK-27199 introduced the use of `ZoneId` instead of `TimeZone` in a few date/time expressions.
There were 3 occurrences of `ctx.addReferenceObj("zoneId", zoneId)` in that PR, which had a bug because while the `java.time.ZoneId` base type is public, the actual concrete implementation classes are not public, so using the 2-arg version of `CodegenContext.addReferenceObj` would incorrectly generate code that reference non-public types (`java.time.ZoneRegion`, to be specific). The 3-arg version should be used, with the class name of the referenced object explicitly specified to the public base type.

One of such occurrences was caught in testing in the main PR of SPARK-27199 (https://github.com/apache/spark/pull/24141), for `DateFormatClass`. But the other 2 occurrences slipped through because there were no test cases that covered them.

Example of this bug in the current Apache Spark master, in a Spark Shell:
```
scala> Seq(("2016-04-08", "yyyy-MM-dd")).toDF("s", "f").repartition(1).selectExpr("to_unix_timestamp(s, f)").show
...
java.lang.IllegalAccessError: tried to access class java.time.ZoneRegion from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1
```

This PR fixes the codegen issues and adds the corresponding unit tests.

## How was this patch tested?

Enhanced tests in `DateExpressionsSuite` for `to_unix_timestamp` and `from_unixtime`.

Closes #24352 from rednaxelafx/fix-spark-27199.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 13:31:18 +08:00
Dilip Biswal 5d8aee5886 [SPARK-27445][SQL][TEST] Update SQLQueryTestSuite to process files ending with .sql
## What changes were proposed in this pull request?
While using vi or vim to edit the test files the .swp or .swo files are created and attempt to run the test suite in the presence of these files causes errors like below :
```
nfo] - subquery/exists-subquery/.exists-basic.sql.swp *** FAILED *** (117 milliseconds)
[info]   java.io.FileNotFoundException: /Users/dbiswal/mygit/apache/spark/sql/core/target/scala-2.12/test-classes/sql-tests/results/subquery/exists-subquery/.exists-basic.sql.swp.out (No such file or directory)
[info]   at java.io.FileInputStream.open0(Native Method)
[info]   at java.io.FileInputStream.open(FileInputStream.java:195)
[info]   at java.io.FileInputStream.<init>(FileInputStream.java:138)
[info]   at org.apache.spark.sql.catalyst.util.package$.fileToString(package.scala:49)
[info]   at org.apache.spark.sql.SQLQueryTestSuite.runQueries(SQLQueryTestSuite.scala:247)
[info]   at org.apache.spark.sql.SQLQueryTestSuite.$anonfun$runTest$11(SQLQueryTestSuite.scala:192)
```
~~This minor pr adds these temp files in the ignore list.~~
While computing the list of test files to process, only consider files with `.sql` extension. This makes sure the unwanted temp files created from various editors are ignored from processing.
## How was this patch tested?
Verified manually.

Closes #24333 from dilipbiswal/dkb_sqlquerytest.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-11 14:50:46 -07:00
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

But mostly, remove use of postfix syntax where it causes warnings without the `scala.language.postfixOps` import. This is mostly in expressions like "120000 milliseconds". Which, I'd like to simplify to things like "2.minutes" anyway.

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
maryannxue 43da473c1c [SPARK-27225][SQL] Implement join strategy hints
## What changes were proposed in this pull request?

This PR extends the existing BROADCAST join hint (for both broadcast-hash join and broadcast-nested-loop join) by implementing other join strategy hints corresponding to the rest of Spark's existing join strategies: shuffle-hash, sort-merge, cartesian-product. The hint names: SHUFFLE_MERGE, SHUFFLE_HASH, SHUFFLE_REPLICATE_NL are partly different from the code names in order to make them clearer to users and reflect the actual algorithms better.

The hinted strategy will be used for the join with which it is associated if it is applicable/doable.

Conflict resolving rules in case of multiple hints:
1. Conflicts within either side of the join: take the first strategy hint specified in the query, or the top hint node in Dataset. For example, in "select /*+ merge(t1) */ /*+ broadcast(t1) */ k1, v2 from t1 join t2 on t1.k1 = t2.k2", take "merge(t1)"; in ```df1.hint("merge").hint("shuffle_hash").join(df2)```, take "shuffle_hash". This is a general hint conflict resolving strategy, not specific to join strategy hint.
2. Conflicts between two sides of the join:
  a) In case of different strategy hints, hints are prioritized as ```BROADCAST``` over ```SHUFFLE_MERGE``` over ```SHUFFLE_HASH``` over ```SHUFFLE_REPLICATE_NL```.
  b) In case of same strategy hints but conflicts in build side, choose the build side based on join type and size.

## How was this patch tested?

Added new UTs.

Closes #24164 from maryannxue/join-hints.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 00:14:37 +08:00
s71955 239082d966 [SPARK-27403][SQL] Fix updateTableStats to update table stats always with new stats or None
## What changes were proposed in this pull request?

System shall update the table stats automatically if user set spark.sql.statistics.size.autoUpdate.enabled as true, currently this property is not having any significance even if it is enabled or disabled. This feature is similar to Hives auto-gather feature where statistics are automatically computed by default if this feature is enabled.
Reference:
https://cwiki.apache.org/confluence/display/Hive/StatsDev

As part of fix , autoSizeUpdateEnabled  validation is been done initially so that system will calculate the table size for the user automatically and record it in metastore as per user expectation.

## How was this patch tested?
UT is written and manually verified in cluster.
Tested with unit tests + some internal tests on real cluster.

Before fix:

![image](https://user-images.githubusercontent.com/12999161/55688682-cd8d4780-5998-11e9-85da-e1a4e34419f6.png)

After fix
![image](https://user-images.githubusercontent.com/12999161/55688654-7d15ea00-5998-11e9-973f-1f4cee27018f.png)

Closes #24315 from sujith71955/master_autoupdate.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-11 08:53:00 -07:00
Gengliang Wang 4177292dcd [SPARK-27435][SQL] Support schema pruning in ORC V2
## What changes were proposed in this pull request?

Currently, the optimization rule `SchemaPruning` only works for Parquet/Orc V1.
We should have the same optimization in ORC V2.

## How was this patch tested?

Unit test

Closes #24338 from gengliangwang/schemaPruningForV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-11 20:03:32 +08:00
chakravarthiT 074533334d [SPARK-27088][SQL] Add a configuration to set log level for each batch at RuleExecutor
## What changes were proposed in this pull request?

Similar to #22406 , which has made log level for plan changes by each rule configurable ,this PR is to make log level for plan changes by each batch configurable,and I have reused the same configuration: "spark.sql.optimizer.planChangeLog.level".

Config proposed in this PR ,
spark.sql.optimizer.planChangeLog.batches - enable plan change logging only for a set of specified batches, separated by commas.

## How was this patch tested?

Added UT , also tested manually and attached screenshots below.

1)Setting spark.sql.optimizer.planChangeLog.leve to warn.

![settingLogLevelToWarn](https://user-images.githubusercontent.com/45845595/54556730-8803dd00-49df-11e9-95ab-ebb0c8d735ef.png)

2)setting spark.sql.optimizer.planChangeLog.batches to Resolution and Subquery.
![settingBatchestoLog](https://user-images.githubusercontent.com/45845595/54556740-8cc89100-49df-11e9-80ab-fbbbe1ff2cdf.png)

3)  plan change logging enabled only for a set of specified batches(Resolution and Subquery)
![batchloggingOp](https://user-images.githubusercontent.com/45845595/54556788-ab2e8c80-49df-11e9-9ae0-57815f552896.png)

Closes #24136 from chakravarthiT/logBatches.

Lead-authored-by: chakravarthiT <45845595+chakravarthiT@users.noreply.github.com>
Co-authored-by: chakravarthiT <tcchakra@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-11 10:02:27 +09:00
ocaballero 181d190c60 [MINOR][SQL] Unnecessary access to externalCatalog
Necessarily access the external catalog without having to do it

## What changes were proposed in this pull request?

The existsFunction function has been changed because it unnecessarily accessed the externalCatalog to find if the database exists in cases where the function is in the functionRegistry

## How was this patch tested?

It has been tested through spark-shell and accessing the metastore logs of hive.

Inside spark-shell we use spark.table (% tableA%). SelectExpr ("trim (% columnA%)") in the current version and it appears every time:

org.apache.hadoop.hive.metastore.HiveMetaStore.audit: cmd = get_database: default

Once the change is made, no record appears

Closes #24312 from OCaballero/master.

Authored-by: ocaballero <oliver.caballero.alvarez@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-11 10:00:09 +09:00
Maxim Gekk ab8710b579 [SPARK-27423][SQL] Cast DATE <-> TIMESTAMP according to the SQL standard
## What changes were proposed in this pull request?

According to SQL standard, value of `DATE` type is union of year, month, dayInMonth, and it is independent from any time zones. To convert it to Catalyst's `TIMESTAMP`, `DATE` value should be "extended" by the time at midnight - `00:00:00`. The resulted local date+time should be considered as a timestamp in the session time zone, and casted to microseconds since epoch in `UTC` accordingly.

The reverse casting from `TIMESTAMP` to `DATE` should be performed in the similar way. `TIMESTAMP` values should be represented as a local date+time in the session time zone. And the time component should be just removed. For example, `TIMESTAMP 2019-04-10 00:10:12` -> `DATE 2019-04-10`. The resulted date is converted to days since epoch `1970-01-01`.

## How was this patch tested?

The changes were tested by existing test suites - `DateFunctionsSuite`, `DateExpressionsSuite` and `CastSuite`.

Closes #24332 from MaxGekk/cast-timestamp-to-date2.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 22:41:19 +08:00
Maxim Gekk 1470f23ec9 [SPARK-27422][SQL] current_date() should return current date in the session time zone
## What changes were proposed in this pull request?

In the PR, I propose to revert 2 commits 06abd06112 and 61561c1c2d, and take current date via `LocalDate.now` in the session time zone. The result is stored as days since epoch `1970-01-01`.

## How was this patch tested?

It was tested by `DateExpressionsSuite`, `DateFunctionsSuite`, `DateTimeUtilsSuite`, and `ComputeCurrentTimeSuite`.

Closes #24330 from MaxGekk/current-date2.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 21:54:50 +08:00
10129659 5ea4deec44 [SPARK-26012][SQL] Null and '' values should not cause dynamic partition failure of string types
Dynamic partition will fail when both '' and null values are taken as dynamic partition values simultaneously.
For example, the test bellow will fail before this PR:

test("Null and '' values should not cause dynamic partition failure of string types") {
withTable("t1", "t2") {
spark.range(3).write.saveAsTable("t1")
spark.sql("select id, cast(case when id = 1 then '' else null end as string) as p" +
" from t1").write.partitionBy("p").saveAsTable("t2")
checkAnswer(spark.table("t2").sort("id"), Seq(Row(0, null), Row(1, null), Row(2, null)))
}
}

The error is: 'org.apache.hadoop.fs.FileAlreadyExistsException: File already exists'.
This PR convert the empty strings to null for partition values.
This is another way for PR(https://github.com/apache/spark/pull/23010)

(Please fill in changes proposed in this fix)

How was this patch tested?
New added test.

Closes #24334 from eatoncys/FileFormatWriter.

Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 19:54:19 +08:00
韩田田00222924 85e5d4f141 [SPARK-24872] Replace taking the $symbol with $sqlOperator in BinaryOperator's toString method
## What changes were proposed in this pull request?

For BinaryOperator's toString method, it's better to use `$sqlOperator` instead of `$symbol`.

## How was this patch tested?

We can test this patch  with unit tests.

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

Closes #21826 from httfighter/SPARK-24872.

Authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 16:58:01 +08:00
Wenchen Fan 2e90574dd0 [SPARK-27414][SQL] make it clear that date type is timezone independent
## What changes were proposed in this pull request?

In SQL standard, date type is a union of the `year`, `month` and `day` fields. It's timezone independent, which means it does not represent a specific point in the timeline.

Spark SQL follows the SQL standard, this PR is to make it clear that date type is timezone independent
1. improve the doc to highlight that date is timezone independent.
2. when converting string to date,  uses the java time API that can directly parse a `LocalDate` from a string, instead of converting `LocalDate` to a `Instant` at UTC first.
3. when converting date to string, uses the java time API that can directly format a `LocalDate` to a string, instead of converting `LocalDate` to a `Instant` at UTC first.

2 and 3 should not introduce any behavior changes.

## How was this patch tested?

existing tests

Closes #24325 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 16:39:28 +08:00
Ryan Blue 58674d54ba [SPARK-27181][SQL] Add public transform API
## What changes were proposed in this pull request?

This adds a public Expression API that can be used to pass partition transformations to data sources.

## How was this patch tested?

Existing tests to validate no regressions. Added transform cases to DDL suite and v1 conversions suite.

Closes #24117 from rdblue/add-public-transform-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 14:30:39 +08:00
Liang-Chi Hsieh 08858f6abc [SPARK-27253][SQL][FOLLOW-UP] Update doc about parent-session configuration priority
## What changes were proposed in this pull request?

The PR #24189 changes the behavior of merging SparkConf. The existing doc is not updated for it. This is a followup of it to update the doc.

## How was this patch tested?

Doc only change.

Closes #24326 from viirya/SPARK-27253-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-10 13:21:21 +09:00
Maxim Gekk 63e4bf42c2 [SPARK-27401][SQL] Refactoring conversion of Timestamp to/from java.sql.Timestamp
## What changes were proposed in this pull request?

In the PR, I propose simpler implementation of `toJavaTimestamp()`/`fromJavaTimestamp()` by reusing existing functions of `DateTimeUtils`. This will allow to:
- Simply implementation of `toJavaTimestamp()`, and handle properly negative inputs.
- Detect `Long` overflow in conversion of milliseconds (`java.sql.Timestamp`) to microseconds (Catalyst's Timestamp).

## How was this patch tested?

By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `CastSuite`. And by new benchmark for export/import timestamps added to `DateTimeBenchmark`:

Before:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             290            335          49         17.2          58.0       1.0X
Collect longs                                      1234           1681         487          4.1         246.8       0.2X
Collect timestamps                                 1718           1755          63          2.9         343.7       0.2X
```

After:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             283            301          19         17.7          56.6       1.0X
Collect longs                                      1048           1087          36          4.8         209.6       0.3X
Collect timestamps                                 1425           1479          56          3.5         285.1       0.2X
```

Closes #24311 from MaxGekk/conv-java-sql-date-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-09 15:42:27 -07:00
Gengliang Wang 3db117e43e [SPARK-27407][SQL] File source V2: Invalidate cache data on overwrite/append
## What changes were proposed in this pull request?

File source V2 currently incorrectly continues to use cached data even if the underlying data is overwritten.
We should follow https://github.com/apache/spark/pull/13566 and fix it by invalidating and refreshes all the cached data (and the associated metadata) for any Dataframe that contains the given data source path.

## How was this patch tested?

Unit test

Closes #24318 from gengliangwang/invalidCache.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-09 09:25:37 -07:00
francis0407 601fac2cb3 [SPARK-27411][SQL] DataSourceV2Strategy should not eliminate subquery
## What changes were proposed in this pull request?

In DataSourceV2Strategy, it seems we eliminate the subqueries by mistake after normalizing filters.
We have a sql with a scalar subquery:

``` scala
val plan = spark.sql("select * from t2 where t2a > (select max(t1a) from t1)")
plan.explain(true)
```

And we get the log info of DataSourceV2Strategy:
```
Pushing operators to csv:examples/src/main/resources/t2.txt
Pushed Filters:
Post-Scan Filters: isnotnull(t2a#30)
Output: t2a#30, t2b#31
```

The `Post-Scan Filters` should contain the scalar subquery, but we eliminate it by mistake.
```
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('t2a > scalar-subquery#56 [])
   :  +- 'Project [unresolvedalias('max('t1a), None)]
   :     +- 'UnresolvedRelation `t1`
   +- 'UnresolvedRelation `t2`

== Analyzed Logical Plan ==
t2a: string, t2b: string
Project [t2a#30, t2b#31]
+- Filter (t2a#30 > scalar-subquery#56 [])
   :  +- Aggregate [max(t1a#13) AS max(t1a)#63]
   :     +- SubqueryAlias `t1`
   :        +- RelationV2[t1a#13, t1b#14] csv:examples/src/main/resources/t1.txt
   +- SubqueryAlias `t2`
      +- RelationV2[t2a#30, t2b#31] csv:examples/src/main/resources/t2.txt

== Optimized Logical Plan ==
Filter (isnotnull(t2a#30) && (t2a#30 > scalar-subquery#56 []))
:  +- Aggregate [max(t1a#13) AS max(t1a)#63]
:     +- Project [t1a#13]
:        +- RelationV2[t1a#13, t1b#14] csv:examples/src/main/resources/t1.txt
+- RelationV2[t2a#30, t2b#31] csv:examples/src/main/resources/t2.txt

== Physical Plan ==
*(1) Project [t2a#30, t2b#31]
+- *(1) Filter isnotnull(t2a#30)
   +- *(1) BatchScan[t2a#30, t2b#31] class org.apache.spark.sql.execution.datasources.v2.csv.CSVScan
```
## How was this patch tested?

ut

Closes #24321 from francis0407/SPARK-27411.

Authored-by: francis0407 <hanmingcong123@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 21:45:46 +08:00
mingbo_pb 3e4cfe9dbc [SPARK-27406][SQL] UnsafeArrayData serialization breaks when two machines have different Oops size
## What changes were proposed in this pull request?
ApproxCountDistinctForIntervals holds the UnsafeArrayData data to initialize endpoints. When the UnsafeArrayData is serialized with Java serialization, the BYTE_ARRAY_OFFSET in memory can change if two machines have different pointer width (Oops in JVM).

This PR fixes this issue by using the same way in https://github.com/apache/spark/pull/9030

## How was this patch tested?
Manual test has been done in our tpcds environment and regarding unit test case has been added as well

Closes #24317 from pengbo/SPARK-27406.

Authored-by: mingbo_pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 15:41:42 +08:00
Hyukjin Kwon f16dfb9129 [SPARK-27328][SQL] Add 'deprecated' in ExpressionDescription for extended usage and SQL doc
## What changes were proposed in this pull request?

This PR proposes to two things:

1. Add `deprecated` field to `ExpressionDescription` so that it can be shown in our SQL function documentation (https://spark.apache.org/docs/latest/api/sql/), and it can be shown via `DESCRIBE FUNCTION EXTENDED`.

2. While I am here, add some more restrictions for `note()` and `since()`. Looks some documentations are broken due to malformed `note`:

    ![Screen Shot 2019-03-31 at 3 00 53 PM](https://user-images.githubusercontent.com/6477701/55285518-a3e88500-53c8-11e9-9e99-41d857794fbe.png)

    It should start with 4 spaces and end with a newline. I added some asserts, and fixed the instances together while I am here. This is technically a breaking change but I think it's too trivial to note somewhere (and we're in Spark 3.0.0).

This PR adds `deprecated` property into `from_utc_timestamp` and `to_utc_timestamp` (it's deprecated as of #24195) as examples of using this field.

Now it shows the deprecation information as below:

- **SQL documentation is shown as below:**

    ![Screen Shot 2019-03-31 at 3 07 31 PM](https://user-images.githubusercontent.com/6477701/55285537-2113fa00-53c9-11e9-9932-f5693a03332d.png)

- **`DESCRIBE FUNCTION EXTENDED from_utc_timestamp;`**:

    ```
    Function: from_utc_timestamp
    Class: org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp
    Usage: from_utc_timestamp(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 03:40:00.0'.
    Extended Usage:
        Examples:
          > SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul');
           2016-08-31 09:00:00

        Since: 1.5.0

        Deprecated:
          Deprecated since 3.0.0. See SPARK-25496.

    ```

## How was this patch tested?

Manually tested via:

- For documentation verification:

    ```
    $ cd sql
    $ sh create-docs.sh
    ```

- For checking description:

    ```
    $ ./bin/spark-sql
    ```
    ```
    spark-sql> DESCRIBE FUNCTION EXTENDED from_utc_timestamp;
    spark-sql> DESCRIBE FUNCTION EXTENDED to_utc_timestamp;
    ```

Closes #24259 from HyukjinKwon/SPARK-27328.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 13:49:42 +08:00
Wenchen Fan 051336d9dd [SPARK-25496][SQL][FOLLOWUP] avoid using to_utc_timestamp
## What changes were proposed in this pull request?

in https://github.com/apache/spark/pull/24195 , we deprecate `from/to_utc_timestamp`.

This PR removes unnecessary use of `to_utc_timestamp` in the test.

## How was this patch tested?

test only PR

Closes #24319 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 10:13:38 +08:00
Gengliang Wang d50603a37c [SPARK-27271][SQL] Migrate Text to File Data Source V2
## What changes were proposed in this pull request?

Migrate Text source to File Data Source V2

## How was this patch tested?

Unit test

Closes #24207 from gengliangwang/textV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 10:15:22 -07:00
Maxim Gekk 00241733a6 [SPARK-27405][SQL][TEST] Restrict the range of generated random timestamps
## What changes were proposed in this pull request?

In the PR, I propose to restrict the range of random timestamp literals generated in `LiteralGenerator. timestampLiteralGen`. The generator creates instances of `java.sql.Timestamp` by passing milliseconds since epoch as `Long` type. Converting the milliseconds to microseconds can cause arithmetic overflow of Long type because Catalyst's Timestamp type stores microseconds since epoch in `Long` type internally as well. Proposed interval of random milliseconds is `[Long.MinValue / 1000, Long.MaxValue / 1000]`.

For example, generated timestamp `new java.sql.Timestamp(-3948373668011580000)` causes `Long` overflow at the method:
```scala
  def fromJavaTimestamp(t: Timestamp): SQLTimestamp = {
  ...
      MILLISECONDS.toMicros(t.getTime()) + NANOSECONDS.toMicros(t.getNanos()) % NANOS_PER_MICROS
  ...
  }
```
because `t.getTime()` returns `-3948373668011580000` which is multiplied by `1000` at `MILLISECONDS.toMicros`, and the result `-3948373668011580000000` is less than `Long.MinValue`.

## How was this patch tested?

By `DateExpressionsSuite` in the PR https://github.com/apache/spark/pull/24311

Closes #24316 from MaxGekk/random-timestamps-gen.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 09:53:00 -07:00
Yuming Wang 33f3c48cac [SPARK-27176][SQL] Upgrade hadoop-3's built-in Hive maven dependencies to 2.3.4
## What changes were proposed in this pull request?

This PR mainly contains:
1. Upgrade hadoop-3's built-in Hive maven dependencies to 2.3.4.
2. Resolve compatibility issues between Hive 1.2.1 and Hive 2.3.4 in the `sql/hive` module.

## How was this patch tested?
jenkins test hadoop-2.7
manual test hadoop-3:
```shell
build/sbt clean package -Phadoop-3.2 -Phive
export SPARK_PREPEND_CLASSES=true

# rm -rf metastore_db

cat <<EOF > test_hadoop3.scala
spark.range(10).write.saveAsTable("test_hadoop3")
spark.table("test_hadoop3").show
EOF

bin/spark-shell --conf spark.hadoop.hive.metastore.schema.verification=false --conf spark.hadoop.datanucleus.schema.autoCreateAll=true -i test_hadoop3.scala
```

Closes #23788 from wangyum/SPARK-23710-hadoop3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-08 08:42:21 -07:00
Michael Allman 215609def2 [SPARK-25407][SQL] Allow nested access for non-existent field for Parquet file when nested pruning is enabled
## What changes were proposed in this pull request?

As part of schema clipping in `ParquetReadSupport.scala`, we add fields in the Catalyst requested schema which are missing from the Parquet file schema to the Parquet clipped schema. However, nested schema pruning requires we ignore unrequested field data when reading from a Parquet file. Therefore we pass two schema to `ParquetRecordMaterializer`: the schema of the file data we want to read and the schema of the rows we want to return. The reader is responsible for reconciling the differences between the two.

Aside from checking whether schema pruning is enabled, there is an additional complication to constructing the Parquet requested schema. The manner in which Spark's two Parquet readers reconcile the differences between the Parquet requested schema and the Catalyst requested schema differ. Spark's vectorized reader does not (currently) support reading Parquet files with complex types in their schema. Further, it assumes that the Parquet requested schema includes all fields requested in the Catalyst requested schema. It includes logic in its read path to skip fields in the Parquet requested schema which are not present in the file.

Spark's parquet-mr based reader supports reading Parquet files of any kind of complex schema, and it supports nested schema pruning as well. Unlike the vectorized reader, the parquet-mr reader requires that the Parquet requested schema include only those fields present in the underlying Parquet file's schema. Therefore, in the case where we use the parquet-mr reader we intersect the Parquet clipped schema with the Parquet file's schema to construct the Parquet requested schema that's set in the `ReadContext`.

_Additional description (by HyukjinKwon):_

Let's suppose that we have a Parquet schema as below:

```
message spark_schema {
  required int32 id;
  optional group name {
    optional binary first (UTF8);
    optional binary last (UTF8);
  }
  optional binary address (UTF8);
}
```

Currently, the clipped schema as follows:

```
message spark_schema {
  optional group name {
    optional binary middle (UTF8);
  }
  optional binary address (UTF8);
}
```

Parquet MR does not support access to the nested non-existent field (`name.middle`).

To workaround this, this PR removes `name.middle` request at all to Parquet reader as below:

```
Parquet requested schema:
message spark_schema {
  optional binary address (UTF8);
}
```

and produces the record (`name.middle`) properly as the requested Catalyst schema.

```
root
-- name: struct (nullable = true)
    |-- middle: string (nullable = true)
-- address: string (nullable = true)
```

I think technically this is what Parquet library should support since Parquet library made a design decision to produce `null` for non-existent fields IIRC. This PR targets to work around it.

## How was this patch tested?

A previously ignored test case which exercises the failure scenario this PR addresses has been enabled.

This closes #22880

Closes #24307 from dongjoon-hyun/SPARK-25407.

Lead-authored-by: Michael Allman <msa@allman.ms>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-08 22:26:02 +09:00
Gengliang Wang 02e9f93309 [SPARK-27384][SQL] File source V2: Prune unnecessary partition columns
## What changes were proposed in this pull request?

When scanning file sources, we can prune unnecessary partition columns on constructing input partitions, so that:
1. Reduce the data transformation from Driver to Executors
2. Make it easier to implement columnar batch readers, since the partition columns are already pruned.
## How was this patch tested?

Existing unit tests.

Closes #24296 from gengliangwang/prunePartitionValue.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-08 15:14:02 +08:00
Yuming Wang 017919b636 [SPARK-27383][SQL][TEST] Avoid using hard-coded jar names in Hive tests
## What changes were proposed in this pull request?

This pr avoid using hard-coded jar names(`hive-contrib-0.13.1.jar` and `hive-hcatalog-core-0.13.1.jar`) in Hive tests. This change makes it easy to change when upgrading the built-in Hive to 2.3.4.

## How was this patch tested?

Existing test

Closes #24294 from wangyum/SPARK-27383.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-06 18:06:52 -05:00
cxzl25 6450c5948a [SPARK-26992][STS] Fix STS scheduler pool correct delivery
## What changes were proposed in this pull request?

The user sets the value of spark.sql.thriftserver.scheduler.pool.
Spark thrift server saves this value in the LocalProperty of threadlocal type, but does not clean up after running, causing other sessions to run in the previously set pool name.

## How was this patch tested?

manual tests

Closes #23895 from cxzl25/thrift_server_scheduler_pool_pollute.

Lead-authored-by: cxzl25 <cxzl25@users.noreply.github.com>
Co-authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-06 17:14:29 -05:00
Jose Torres 4a5768b2a2 [SPARK-27391][SS] Don't initialize a lazy val in ContinuousExecution job.
## What changes were proposed in this pull request?

Fix a potential deadlock in ContinuousExecution by not initializing the toRDD lazy val.

Closes #24301 from jose-torres/deadlock.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Jose Torres <torres.joseph.f+github@gmail.com>
2019-04-05 12:56:36 -07:00
gengjiaan 979bb905b7 [SPARK-26936][SQL] Fix bug of insert overwrite local dir can not create temporary path in local staging directory
## What changes were proposed in this pull request?
Th environment of my cluster as follows:
```
OS:Linux version 2.6.32-220.7.1.el6.x86_64 (mockbuildc6b18n3.bsys.dev.centos.org) (gcc version 4.4.6 20110731 (Red Hat 4.4.6-3) (GCC) ) #1 SMP Wed Mar 7 00:52:02 GMT 2012
Hadoop: 2.7.2
Spark: 2.3.0 or 3.0.0(master branch)
Hive: 1.2.1
```

My spark run on deploy mode yarn-client.

If I execute the SQL `insert overwrite local directory '/home/test/call_center/' select * from call_center`, a HiveException will appear as follows:
`Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: Mkdirs failed to create file:/home/xitong/hive/stagingdir_hive_2019-02-19_17-31-00_678_1816816774691551856-1/-ext-10000/_temporary/0/_temporary/attempt_20190219173233_0002_m_000000_3 (exists=false, cwd=file:/data10/yarn/nm-local-dir/usercache/xitong/appcache/application_1543893582405_6126857/container_e124_1543893582405_6126857_01_000011)
at org.apache.hadoop.hive.ql.io.HiveFileFormatUtils.getHiveRecordWriter(HiveFileFormatUtils.java:249)`
Current spark sql generate a local temporary path in local staging directory.The schema of local temporary path start with `file`, so the HiveException appears.
This PR change the local temporary path to HDFS temporary path, and use DistributedFileSystem instance copy the data from HDFS temporary path to local directory.
If Spark run on local deploy mode, 'insert overwrite local directory' works fine.
## How was this patch tested?

UT cannot support yarn-client mode.The test is in my product environment.

Closes #23841 from beliefer/fix-bug-of-insert-overwrite-local-dir.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 14:02:46 -05:00
Dongjoon Hyun 982c4c8e3c [SPARK-27390][CORE][SQL][TEST] Fix package name mismatch
## What changes were proposed in this pull request?

This PR aims to clean up package name mismatches.

## How was this patch tested?

Pass the Jenkins.

Closes #24300 from dongjoon-hyun/SPARK-27390.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-05 11:50:37 -07:00
gatorsmile 5678e687c6 [SPARK-27393][SQL] Show ReusedSubquery in the plan when the subquery is reused
## What changes were proposed in this pull request?
With this change, we can easily identify the plan difference when subquery is reused.

When the reuse is enabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery240 + ReusedSubquery Subquery subquery240) AS (scalarsubquery() + scalarsubquery())#253]
   :  :- Subquery subquery240
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#250])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#256, count#257L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- ReusedSubquery Subquery subquery240
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

When the reuse is disabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery286 + Subquery subquery287) AS (scalarsubquery() + scalarsubquery())#299]
   :  :- Subquery subquery286
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#296])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#302, count#303L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- Subquery subquery287
   :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#298])
   :        +- Exchange SinglePartition
   :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#306, count#307L])
   :              +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :                 +- Scan[obj#12]
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

## How was this patch tested?
Modified the existing test.

Closes #24258 from gatorsmile/followupSPARK-27279.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-05 08:31:41 -07:00
Gengliang Wang 568db94e0c [SPARK-27356][SQL] File source V2: Fix the case that data columns overlap with partition schema
## What changes were proposed in this pull request?

In the current file source V2 framework, the schema of `FileScan` is not returned correctly if there are overlap columns between `dataSchema` and `partitionSchema`. The actual schema should be
`dataSchema - overlapSchema + partitionSchema`, which might have different column order from the pushed down `requiredSchema` in `SupportsPushDownRequiredColumns.pruneColumns`.

For example, if the data schema is `[a: String, b: String, c: String]` and the partition schema is `[b: Int, d: Int]`, the result schema is `[a: String, b: Int, c: String, d: Int]` in current `FileTable` and `HadoopFsRelation`. while the actual scan schema is `[a: String, c: String, b: Int, d: Int]` in `FileScan`.

To fix the corner case, this PR proposes that the output schema of `FileTable` should be `dataSchema - overlapSchema + partitionSchema`, so that the column order is consistent with `FileScan`.
Putting all the partition columns to the end of table schema is more reasonable.

## How was this patch tested?

Unit test.

Closes #24284 from gengliangwang/FixReadSchema.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-05 13:34:46 +08:00
Aayushmaan Jain 04e53d2e3c [SPAR-27342][SQL] Optimize Limit 0 queries
## What changes were proposed in this pull request?
With this change, unnecessary file scans are avoided in case of Limit 0 queries.

I added a case (rule) to `PropagateEmptyRelation` to replace `GlobalLimit 0` and `LocalLimit 0` nodes with an empty `LocalRelation`. This prunes the subtree under the Limit 0 node and further allows other rules of `PropagateEmptyRelation` to optimize the Logical Plan - while remaining semantically consistent with the Limit 0 query.

For instance:
**Query:**
`SELECT * FROM table1 INNER JOIN (SELECT * FROM table2 LIMIT 0) AS table2 ON table1.id = table2.id`

**Optimized Plan without fix:**
```
Join Inner, (id#79 = id#87)
:- Filter isnotnull(id#79)
:  +- Relation[id#79,num1#80] parquet
+- Filter isnotnull(id#87)
   +- GlobalLimit 0
      +- LocalLimit 0
         +- Relation[id#87,num2#88] parquet
```

**Optimized Plan with fix:**
`LocalRelation <empty>, [id#75, num1#76, id#77, num2#78]`

## How was this patch tested?
Added unit tests to verify Limit 0 optimization for:
- Simple query containing Limit 0
- Inner Join, Left Outer Join, Right Outer Join, Full Outer Join queries containing Limit 0 as one of their children
- Nested Inner Joins between 3 tables with one of them having a Limit 0 clause.
- Intersect query wherein one of the subqueries was a Limit 0 query.

Closes #24271 from aayushmaanjain/optimize-limit0.

Authored-by: Aayushmaan Jain <aayushmaan.jain42@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-04 21:19:40 -07:00
Ruben Fiszel 0e44a51f2e [SPARK-24345][SQL] Improve ParseError stop location when offending symbol is a token
In the case where the offending symbol is a CommonToken, this PR increases the accuracy of the start and stop origin by leveraging the start and stop index information from CommonToken.

Closes #21334 from rubenfiszel/patch-1.

Lead-authored-by: Ruben Fiszel <rubenfiszel@gmail.com>
Co-authored-by: rubenfiszel <rfiszel@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-04 18:20:34 -05:00
Dongjoon Hyun 938d954375 [SPARK-27382][SQL][TEST] Update Spark 2.4.x testing in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

Since Apache Spark 2.4.1 vote passed and is distributed into mirrors, we need to test 2.4.1. This should land on both `master` and `branch-2.4`.

## How was this patch tested?

Pass the Jenkins.

Closes #24292 from dongjoon-hyun/SPARK-27382.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-04 13:49:56 -07:00
Wenchen Fan f7bd1ab586 [SPARK-26811][SQL][FOLLOWUP] some more document fixes
## What changes were proposed in this pull request?

while working on https://github.com/apache/spark/pull/24129, I realized that I missed some document fixes in https://github.com/apache/spark/pull/24285. This PR covers all of them.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #24295 from cloud-fan/doc.
2019-04-05 01:07:08 +08:00
Yuming Wang 1d95dea307 [SPARK-27349][SQL] Dealing with TimeVars removed in Hive 2.x
## What changes were proposed in this pull request?
`hive.stats.jdbc.timeout` and `hive.stats.retries.wait` were removed by [HIVE-12164](https://issues.apache.org/jira/browse/HIVE-12164).
This pr to deal with this change.

## How was this patch tested?

unit tests

Closes #24277 from wangyum/SPARK-27349.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-03 22:52:37 -07:00
Wenchen Fan 5c50f68253 [SPARK-26811][SQL][FOLLOWUP] fix some documentation
## What changes were proposed in this pull request?

It's a followup of https://github.com/apache/spark/pull/24012 , to fix 2 documentation:
1. `SupportsRead` and `SupportsWrite` are not internal anymore. They are public interfaces now.
2. `Scan` should link the `BATCH_READ` instead of hardcoding it.

## How was this patch tested?
N/A

Closes #24285 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 10:31:27 +08:00
Dongjoon Hyun b51763612a Revert "[SPARK-27278][SQL] Optimize GetMapValue when the map is a foldable and the key is not"
This reverts commit 5888b15d9c.
2019-04-03 09:41:13 -07:00
Wenchen Fan ffb362a705 [SPARK-19712][SQL][FOLLOW-UP] reduce code duplication
## What changes were proposed in this pull request?

abstract some common code into a method.

## How was this patch tested?

existing tests

Closes #24281 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 00:37:57 +08:00
Liang-Chi Hsieh d04a7371da [MINOR][DOC][SQL] Remove out-of-date doc about ORC in DataFrameReader and Writer
## What changes were proposed in this pull request?

According to current status, `orc` is available even Hive support isn't enabled. This is a minor doc change to reflect it.

## How was this patch tested?

Doc only change.

Closes #24280 from viirya/fix-orc-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-03 09:11:09 -07:00
Maxim Gekk 1bc672366d [SPARK-27344][SQL][TEST] Support the LocalDate and Instant classes in Java Bean encoders
## What changes were proposed in this pull request?

- Added new test for Java Bean encoder of the classes: `java.time.LocalDate` and `java.time.Instant`.
- Updated comment for `Encoders.bean`
- New Row getters: `getLocalDate` and `getInstant`
- Extended `inferDataType` to infer types for `java.time.LocalDate` -> `DateType` and `java.time.Instant` -> `TimestampType`.

## How was this patch tested?

By `JavaBeanDeserializationSuite`

Closes #24273 from MaxGekk/bean-instant-localdate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 17:45:59 +08:00
Dilip Biswal 3286bff942 [SPARK-27255][SQL] Report error when illegal expressions are hosted by a plan operator.
## What changes were proposed in this pull request?
In the PR, we raise an AnalysisError when we detect the presense of aggregate expressions in where clause. Here is the problem description from the JIRA.

Aggregate functions should not be allowed in WHERE clause. But Spark SQL throws an exception when generating codes. It is supposed to throw an exception during parsing or analyzing.

Here is an example:
```
val df = spark.sql("select * from t where sum(ta) > 0")
df.explain(true)
df.show()
```
Resulting exception:
```
Exception in thread "main" java.lang.UnsupportedOperationException: Cannot generate code for expression: sum(cast(input[0, int, false] as bigint))
	at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode(Expression.scala:291)
	at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode$(Expression.scala:290)
	at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:138)
	at scala.Option.getOrElse(Option.scala:138)
```
Checked the behaviour of other database and all of them return an exception:
**Postgress**
```
select * from foo where max(c1) > 0;
Error
ERROR: aggregate functions are not allowed in WHERE Position: 25
```
**DB2**
```
db2 => select * from foo where max(c1) > 0;
SQL0120N  Invalid use of an aggregate function or OLAP function.
```
**Oracle**
```
select * from foo where max(c1) > 0;
ORA-00934: group function is not allowed here
```
**MySql**
```
select * from foo where max(c1) > 0;
Invalid use of group function
```

**Update**
This PR has been enhanced to report error when expressions such as Aggregate, Window, Generate are hosted by operators where they are invalid.
## How was this patch tested?
Added tests in AnalysisErrorSuite and group-by.sql

Closes #24209 from dilipbiswal/SPARK-27255.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 13:05:06 +08:00
Maxim Gekk 1d20d13149 [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 10:55:56 +08:00
Dilip Biswal b8b5acdd41 [SPARK-19712][SQL][FOLLOW-UP] Don't do partial pushdown when pushing down LeftAnti joins below Aggregate or Window operators.
## What changes were proposed in this pull request?
After [23750](https://github.com/apache/spark/pull/23750), we may pushdown left anti joins below aggregate and window operators with a partial join condition. This is not correct and was pointed out by hvanhovell and cloud-fan [here](https://github.com/apache/spark/pull/23750#discussion_r270017097). This pr addresses their comments.
## How was this patch tested?
Added two new tests to verify the behaviour.

Closes #24253 from dilipbiswal/SPARK-19712-followup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 09:56:27 +08:00
Hyukjin Kwon d7dd59a6b4 [SPARK-26224][SQL][PYTHON][R][FOLLOW-UP] Add notes about many projects in withColumn at SparkR and PySpark as well
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23285. This PR adds the notes into PySpark and SparkR documentation as well.

While I am here, I revised the doc a bit to make it sound a bit more neutral

## How was this patch tested?

Manually built the doc and verified.

Closes #24272 from HyukjinKwon/SPARK-26224.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-03 08:30:24 +09:00
Hyukjin Kwon 949d712839 [SPARK-27346][SQL] Loosen the newline assert condition on 'examples' field in ExpressionInfo
## What changes were proposed in this pull request?

I haven't tested by myself on Windows and I am not 100% sure if this is going to cause an actual problem.

However, this one line:

827383a97c/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java (L82)

made me to investigate a lot today.

Given my speculation, if Spark is built in Linux and it's executed on Windows, it looks possible for multiline strings, like,

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala (L146-L150)

to throw an exception because the newline in the binary is `\n` but `System.lineSeparator` returns `\r\n`.

I think this is not yet found because this particular codes are not released yet (see SPARK-26426).

Looks just better to loosen the condition and forget about this stuff.

This should be backported into branch-2.4 as well.

## How was this patch tested?

N/A

Closes #24274 from HyukjinKwon/SPARK-27346.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-03 08:27:41 +09:00
Sean Owen d4420b455a [SPARK-27323][CORE][SQL][STREAMING] Use Single-Abstract-Method support in Scala 2.12 to simplify code
## What changes were proposed in this pull request?

Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.

## How was this patch tested?

Existing tests.

Closes #24241 from srowen/SPARK-27323.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-02 07:37:05 -07:00
Dongjoon Hyun d575a453db Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp"
This reverts commit c5e83ab92c.
2019-04-02 01:05:54 -07:00
Marco Gaido 0b150f833c [SPARK-26224][SQL] Advice the user when creating many project on subsequent calls to withColumn
## What changes were proposed in this pull request?

We have seen many cases when users make several subsequent calls to `withColumn` on a Dataset. This leads now to the generation of a lot of `Project` nodes on the top of the plan, with serious problem which can lead also to `StackOverflowException`s.

The PR improves the doc of `withColumn`, in order to advise the user to avoid this pattern and do something different, ie. a single select with all the column he/she needs.

## How was this patch tested?

NA

Closes #23285 from mgaido91/SPARK-26224.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-02 14:12:47 +09:00
Maxim Gekk c5e83ab92c [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-02 10:20:06 +08:00
Liang-Chi Hsieh eaf008ad0e [SPARK-27329][SQL] Pruning nested field in map of map key and value from object serializers
## What changes were proposed in this pull request?

If object serializer has map of map key/value, pruning nested field should work.

Previously object serializer pruner don't recursively prunes nested fields if it is deeply located in map key or value. This patch proposed to address it by slightly factoring the pruning logic.

## How was this patch tested?

Added tests.

Closes #24260 from viirya/SPARK-27329.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-01 13:53:55 -07:00
Marco Gaido 5888b15d9c [SPARK-27278][SQL] Optimize GetMapValue when the map is a foldable and the key is not
## What changes were proposed in this pull request?

When `GetMapValue` contains a foldable Map and a non-foldable key, `SimplifyExtractValueOps` fails to optimize it transforming it into case when statements.
The PR adds a case for covering this situation too.

## How was this patch tested?

added UT

Closes #24223 from mgaido91/SPARK-27278.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-01 09:09:06 -07:00
Maxim Gekk d332958109 [SPARK-27325][SQL] Add implicit encoders for LocalDate and Instant
## What changes were proposed in this pull request?

Added implicit encoders for the `java.time.LocalDate` and `java.time.Instant` classes. This allows creation of datasets from instances of the types.

## How was this patch tested?

Added new tests to `JavaDatasetSuite` and `DatasetSuite`.

Closes #24249 from MaxGekk/instant-localdate-encoders.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-01 23:02:48 +08:00
Marco Gaido 8012f55a9b [SPARK-26812][SQL] Report correct nullability for complex datatypes in Union
## What changes were proposed in this pull request?

When there is a `Union`, the reported output datatypes are the ones of the first plan and the nullability is updated according to all the plans. For complex types, though, the nullability of their elements is not updated using the types from the other plans. This means that the nullability of the inner elements is the one of the first plan. If this is not compatible with the one of other plans, errors can happen (as reported in the JIRA).

The PR proposes to update the nullability of the inner elements of complex datatypes according to most permissive value of all the plans.

## How was this patch tested?

added UT

Closes #23726 from mgaido91/SPARK-26812.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-01 22:22:10 +08:00
chakravarthiT fc9aad0957 [SPARK-27253][SQL] Prioritizes parent session's SQLConf over SparkConf when cloning a session
## What changes were proposed in this pull request?

Cloned session should prioritize `SQLConf` from parent's over `SparkConf`. Currently, when cloning a session, the child session has configuration set in `SparkConf` even the same properties are set to its parent `SQLConf`.

Currently, when a Spark session is cloned, `mergeSparkConf` in `BaseSessionStateBuilder`'s `conf` overwrites  `SQLConf` values as set in `SparkConf`.

This PR proposes to call `mergeSparkConf` only when the parent session is empty.

See below codes to read.

1. Parent's `sessionState`

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L268)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L157-L161)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L88-L90)

2. Child `sessionState`

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L269)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala (L155)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala (L102)

c26379b446/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala (L74)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L305)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L283)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L292)

5dab5f651f/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala (L88-L90)

## How was this patch tested?
Added UT and with existing Unit Tests.

Closes #24189 from chakravarthiT/CloneDiscardsConf.

Authored-by: chakravarthiT <tcchakra@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-01 09:33:18 +09:00
Takeshi Yamamuro 885aab40a2 [SPARK-27266][SQL] Support ANALYZE TABLE to collect tables stats for cached catalog views
## What changes were proposed in this pull request?
The current master doesn't support ANALYZE TABLE to collect tables stats for catalog views even if they are cached as follows;

```scala
scala> sql(s"CREATE VIEW v AS SELECT 1 c")
scala> sql(s"CACHE LAZY TABLE v")
scala> sql(s"ANALYZE TABLE v COMPUTE STATISTICS")
org.apache.spark.sql.AnalysisException: ANALYZE TABLE is not supported on views.;
...
```

Since SPARK-25196 has supported to an ANALYZE command to collect column statistics for cached catalog view, we could support table stats, too.

## How was this patch tested?
Added tests in `StatisticsCollectionSuite` and `InMemoryColumnarQuerySuite`.

Closes #24200 from maropu/SPARK-27266.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-31 17:24:21 -07:00
Maxim Gekk 6115a5e1a0 [SPARK-27327][SQL] New JSON benchmarks: functions, Dataset[String]
## What changes were proposed in this pull request?

Added new benchmarks for:
1. JSON functions: `from_json`, `json_tuple` and `get_json_object`
2. Parsing `Dataset[String]` with JSON records
3. Comparing just splitting input text by lines with schema inferring, per-line parsing when encoding is set and not set.

Also existing benchmarks were refactored to use the `NoOp` datasource to eliminate overhead of triggers like `.filter((_: Row) => true).count()`.

## How was this patch tested?

By running `JSONBenchmark` locally.

Closes #24252 from MaxGekk/json-benchmark-func.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-01 08:33:16 +09:00
Gengliang Wang 5dab5f651f [SPARK-27326][SQL] Fall back all v2 file sources in InsertIntoTable to V1 FileFormat
## What changes were proposed in this pull request?

In the first PR for file source V2, there was a rule for falling back Orc V2 table to OrcFileFormat: https://github.com/apache/spark/pull/23383/files#diff-57e8244b6964e4f84345357a188421d5R34

As we are migrating more file sources to data source V2, we should make the rule more generic. This PR proposes to:
1. Rename the rule `FallbackOrcDataSourceV2 ` to `FallBackFileSourceV2`.The name is more generic. And we use "fall back" as verb, while "fallback" is noun.
2. Rename the method `fallBackFileFormat` in `FileDataSourceV2` to `fallbackFileFormat`. Here we should use "fallback" as noun.
3. Add new method `fallbackFileFormat` in `FileTable`. This is for falling back to V1 in rule `FallbackOrcDataSourceV2 `.

## How was this patch tested?

Existing Unit tests.

Closes #24251 from gengliangwang/fallbackV1Rule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-30 14:38:26 -07:00
10129659 144b35fe3a [SPARK-27320][SQL] Replacing index with iterator to traverse the expressions list in AggregationIterator, which make it simpler
## What changes were proposed in this pull request?
In AggregationIterator's loop function, we access the expressions by `expressions(i)`, the type of `expressions` is `::`, a subtype of list.

```
while (i < expressionsLength) {
      val func = expressions(i).aggregateFunction
```

This PR replacing  index with iterator to access the expressions list, which make it simpler.

## How was this patch tested?
Existing tests.

Closes #24238 from eatoncys/array.

Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-30 02:27:12 -05:00
Takuya UESHIN f176dd3f28 [SPARK-27314][SQL] Deduplicate exprIds for Union.
## What changes were proposed in this pull request?

We have been having a potential problem with `Union` when the children have the same expression id in their outputs, which happens when self-union.

## How was this patch tested?

Modified some tests to adjust plan changes.

Closes #24236 from ueshin/issues/SPARK-27314/dedup_union.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-29 14:05:38 -07:00
Maxim Gekk 61561c1c2d [SPARK-27252][SQL][FOLLOWUP] Calculate min and max days independently from time zone in ComputeCurrentTimeSuite
## What changes were proposed in this pull request?

This fixes the `analyzer should replace current_date with literals` test in `ComputeCurrentTimeSuite` by making calculation of `min` and `max` days independent from time zone.

## How was this patch tested?

by `ComputeCurrentTimeSuite`.

Closes #24240 from MaxGekk/current-date-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-29 14:28:36 -05:00
Maxim Gekk 06abd06112 [SPARK-27252][SQL] Make current_date() independent from time zones
## What changes were proposed in this pull request?

This makes the `CurrentDate` expression and `current_date` function independent from time zone settings. New result is number of days since epoch in `UTC` time zone. Previously, Spark shifted the current date (in `UTC` time zone) according the session time zone which violets definition of `DateType` - number of days since epoch (which is an absolute point in time, midnight of Jan 1 1970 in UTC time).

The changes makes `CurrentDate` consistent to `CurrentTimestamp` which is independent from time zone too.

## How was this patch tested?

The changes were tested by existing test suites like `DateExpressionsSuite`.

Closes #24185 from MaxGekk/current-date.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-28 18:44:08 -07:00
Xianyang Liu 50cded590f [MINOR] Move java file to java directory
## What changes were proposed in this pull request?

move
```scala
org.apache.spark.sql.execution.streaming.BaseStreamingSource
org.apache.spark.sql.execution.streaming.BaseStreamingSink
```
to java directory

## How was this patch tested?

Existing UT.

Closes #24222 from ConeyLiu/move-scala-to-java.

Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-28 12:11:00 -05:00
zhoukang 43bf4ae641 [SPARK-26914][SQL] Fix scheduler pool may be unpredictable when we only want to use default pool and do not set spark.scheduler.pool for the session
## What changes were proposed in this pull request?

When using fair scheduler mode for thrift server, we may have unpredictable result.
```
val pool = sessionToActivePool.get(parentSession.getSessionHandle)
if (pool != null) {
   sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)
}
```
The cause is we use thread pool to execute queries for thriftserver, and when we call setLocalProperty we may have unpredictab behavior.

```
/**
   * Set a local property that affects jobs submitted from this thread, such as the Spark fair
   * scheduler pool. User-defined properties may also be set here. These properties are propagated
   * through to worker tasks and can be accessed there via
   * [[org.apache.spark.TaskContext#getLocalProperty]].
   *
   * These properties are inherited by child threads spawned from this thread. This
   * may have unexpected consequences when working with thread pools. The standard java
   * implementation of thread pools have worker threads spawn other worker threads.
   * As a result, local properties may propagate unpredictably.
   */
  def setLocalProperty(key: String, value: String) {
    if (value == null) {
      localProperties.get.remove(key)
    } else {
      localProperties.get.setProperty(key, value)
    }
  }
```

I post an example on https://jira.apache.org/jira/browse/SPARK-26914 .

## How was this patch tested?
UT

Closes #23826 from caneGuy/zhoukang/fix-scheduler-error.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-28 09:24:16 -05:00
Gengliang Wang 49b0411549 [SPARK-27291][SQL] PartitioningAwareFileIndex: Filter out empty files on listing files
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/23130, all empty files are excluded from target file splits in `FileSourceScanExec`.
In File source V2, we should keep the same behavior.

This PR suggests to filter out empty files on listing files in `PartitioningAwareFileIndex` so that the upper level doesn't need to handle them.
## How was this patch tested?

Unit test

Closes #24227 from gengliangwang/ignoreEmptyFile.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-27 10:08:38 -07:00
Daoyuan Wang f1fe805bed [SPARK-27279][SQL] Reuse subquery should compare child plan of SubqueryExec
## What changes were proposed in this pull request?

For now, `ReuseSubquery` in Spark compares two subqueries at `SubqueryExec` level, which invalidates the `ReuseSubquery` rule. This pull request fixes this, and add a configuration key for subquery reuse exclusively.

## How was this patch tested?

add a unit test.

Closes #24214 from adrian-wang/reuse.

Authored-by: Daoyuan Wang <me@daoyuan.wang>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-27 08:45:22 -07:00
Takeshi Yamamuro 956b52b167 [SPARK-26771][SQL][FOLLOWUP] Make all the uncache operations non-blocking by default
## What changes were proposed in this pull request?
To make the blocking behaviour consistent, this pr made catalog table/view `uncacheQuery` non-blocking by default. If this pr merged, all the behaviours in spark are non-blocking by default.

## How was this patch tested?
Pass Jenkins.

Closes #24212 from maropu/SPARK-26771-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 21:01:36 +09:00
Liang-Chi Hsieh 93ff69003b [SPARK-27288][SQL] Pruning nested field in complex map key from object serializers
## What changes were proposed in this pull request?

In the original PR #24158, pruning nested field in complex map key was not supported, because some methods in schema pruning did't support it at that moment. This is a followup to add it.

## How was this patch tested?

Added tests.

Closes #24220 from viirya/SPARK-26847-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 19:40:14 +09:00
liuxian fac31104f6 [SPARK-27083][SQL] Add a new conf to control subqueryReuse
## What changes were proposed in this pull request?
Subquery Reuse and Exchange Reuse are not the same feature, if we don't want to reuse subqueries,and we just want to reuse exchanges,only one configuration that cannot be done.

This PR adds a new configuration `spark.sql.subquery.reuse` to control subqueryReuse.

## How was this patch tested?

N/A

Closes #23998 from 10110346/SUBQUERY_REUSE.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 23:37:58 -07:00
Gengliang Wang 6bcd4805d2 [SPARK-27286][SQL] Handles exceptions on proceeding to next record in FilePartitionReader
## What changes were proposed in this pull request?

In data source V2, the method `PartitionReader.next()` has side effects. When the method is called, the current reader proceeds to the next record.
This might throw RuntimeException/IOException and File source V2 framework should handle these exceptions.

## How was this patch tested?

Unit test.

Closes #24225 from gengliangwang/corruptFile.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 22:33:34 -07:00
Yuming Wang ca1433b94a [SPARK-27182][SQL] Move the conflict source code of the sql/core module to sql/core/v1.2.1
## What changes were proposed in this pull request?
To make https://github.com/apache/spark/pull/23788 easy to review. This PR moves `OrcColumnVector.java`, `OrcShimUtils.scala`, `OrcFilters.scala` and `OrcFilterSuite.scala` to `sql/core/v1.2.1` and copies it to `sql/core/v2.3.4`.

## How was this patch tested?

manual tests
```shell
diff -urNa sql/core/v1.2.1 sql/core/v2.3.4
```

Closes #24119 from wangyum/SPARK-27182.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-26 22:32:03 -07:00
Dilip Biswal 6c0e13b456 [SPARK-27285] Support describing output of CTE
## What changes were proposed in this pull request?
SPARK-26982 allows users to describe output of a query. However, it had a limitation of not supporting CTEs due to limitation of the grammar having a single rule to parse both select and inserts. After SPARK-27209, which splits select and insert parsing to two different rules, we can now support describing output of the CTEs easily.

## How was this patch tested?
Existing tests were modified.

Closes #24224 from dilipbiswal/describe_support_cte.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 16:00:56 -07:00
Gengliang Wang 267160b360 [SPARK-27269][SQL] File source v2 should validate data schema only
## What changes were proposed in this pull request?

Currently, File source v2 allows each data source to specify the supported data types by implementing the method `supportsDataType` in `FileScan` and `FileWriteBuilder`.

However, in the read path, the validation checks all the data types in `readSchema`, which might contain partition columns. This is actually a regression. E.g. Text data source only supports String data type, while the partition columns can still contain Integer type since partition columns are processed by Spark.

This PR is to:
1. Refactor schema validation and check data schema only.
2. Filter the partition columns in data schema if user specified schema provided.

## How was this patch tested?

 Unit test

Closes #24203 from gengliangwang/schemaValidation.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-27 07:58:31 +09:00
Maxim Gekk 69035684d4 [SPARK-27242][SQL] Make formatting TIMESTAMP/DATE literals independent from the default time zone
## What changes were proposed in this pull request?

In the PR, I propose to use the SQL config `spark.sql.session.timeZone` in formatting `TIMESTAMP` literals, and make formatting `DATE` literals independent from time zone. The changes make parsing and formatting `TIMESTAMP`/`DATE` literals consistent, and independent from the default time zone of current JVM.

Also this PR ports `TIMESTAMP`/`DATE` literals formatting on Proleptic Gregorian Calendar via using `TimestampFormatter`/`DateFormatter`.

## How was this patch tested?

Added new tests to `LiteralExpressionSuite`

Closes #24181 from MaxGekk/timezone-aware-literals.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 15:29:59 -07:00
Takuya UESHIN 529a061168 [SPARK-26103][SQL][FOLLOW-UP] Use string-interpolation to show the config key.
## What changes were proposed in this pull request?

This is a follow-up of #23169.
We should've used string-interpolation to show the config key in the warn message.

## How was this patch tested?

Existing tests.

Closes #24217 from ueshin/issues/SPARK-26103/s.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-26 20:56:00 +09:00
Dilip Biswal 9cc925cda2 [SPARK-27209][SQL] Split parsing of SELECT and INSERT into two top-level rules in the grammar file.
## What changes were proposed in this pull request?
Currently in the grammar file the rule `query` is responsible to parse both select and insert statements. As a result, we need to have more semantic checks in the code to guard against in-valid insert constructs in a query. Couple of examples are in the `visitCreateView` and `visitAlterView` functions. One other issue is that, we don't catch the `invalid insert constructs` in all the places until checkAnalysis (the errors we raise can be confusing as well). Here are couple of examples :

```SQL
select * from (insert into bar values (2));
```
```
Error in query: unresolved operator 'Project [*];
'Project [*]
+- SubqueryAlias `__auto_generated_subquery_name`
   +- InsertIntoHiveTable `default`.`bar`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, false, false, [c1]
      +- Project [cast(col1#18 as int) AS c1#20]
         +- LocalRelation [col1#18]
```

```SQL
select * from foo where c1 in (insert into bar values (2))
```
```
Error in query: cannot resolve '(default.foo.`c1` IN (listquery()))' due to data type mismatch:
The number of columns in the left hand side of an IN subquery does not match the
number of columns in the output of subquery.
#columns in left hand side: 1.
#columns in right hand side: 0.

Left side columns:
[default.foo.`c1`].
Right side columns:
[].;;
'Project [*]
+- 'Filter c1#6 IN (list#5 [])
   :  +- InsertIntoHiveTable `default`.`bar`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, false, false, [c1]
   :     +- Project [cast(col1#7 as int) AS c1#9]
   :        +- LocalRelation [col1#7]
   +- SubqueryAlias `default`.`foo`
      +- HiveTableRelation `default`.`foo`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#6]
```

For both the cases above, we should reject the syntax at parser level.

In this PR, we create two top-level parser rules to parse `SELECT` and `INSERT` respectively.
I will create a small PR to allow CTEs in DESCRIBE QUERY after this PR is in.
## How was this patch tested?
Added tests to PlanParserSuite and removed the semantic check tests from SparkSqlParserSuites.

Closes #24150 from dilipbiswal/split-query-insert.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-25 17:43:03 -07:00
Yuming Wang 300ec1a74c [SPARK-27226][SQL] Reduce the code duplicate when upgrading built-in Hive
## What changes were proposed in this pull request?

This pr related to #24119. Reduce the code duplicate when upgrading built-in Hive.
To achieve this, we should avoid using classes in `org.apache.orc.storage.*` because these classes will be replaced with `org.apache.hadoop.hive.*` after upgrading the built-in Hive. Such as:
![image](https://user-images.githubusercontent.com/5399861/54437594-e9be1000-476f-11e9-8878-3b7414871ee5.png)

- Move the usage of `org.apache.orc.storage.*` to `OrcShimUtils`:
1. Add wrapper for `VectorizedRowBatch`(Reduce code duplication of [OrcColumnarBatchReader](https://github.com/apache/spark/pull/24166/files#diff-e594f7295e5408c01ace8175166313b6)).
2. Move some serializer/deserializer method out of `OrcDeserializer` and `OrcSerializer`(Reduce code duplication of [OrcDeserializer](https://github.com/apache/spark/pull/24166/files#diff-b933819e6dcaff41eee8fce1e8f2932c) and [OrcSerializer](https://github.com/apache/spark/pull/24166/files#diff-6d3849d88929f6ea25c436d71da729da)).
3. Defined two type aliases: `Operator` and `SearchArgument`(Reduce code duplication of [OrcV1FilterSuite](https://github.com/apache/spark/pull/24166/files#diff-48c4fc7a3b3384a6d0aab246723a0058)).

- Move duplication code to super class:
1. Add a trait for `OrcFilters`(Reduce code duplication of [OrcFilters](https://github.com/apache/spark/pull/24166/files#diff-224b8cbedf286ecbfdd092d1e2e2f237)).
2. Move `checkNoFilterPredicate` from `OrcFilterSuite` to `OrcTest`(Reduce code duplication of [OrcFilterSuite](https://github.com/apache/spark/pull/24166/files#diff-8e05c1faaaec98edd7723e62f84066f1)).

After this pr. We only need to copy these 4 files: OrcColumnVector, OrcFilters, OrcFilterSuite and OrcShimUtils.

## How was this patch tested?

existing tests

Closes #24166 from wangyum/SPARK-27226.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 19:39:00 -05:00
sandeep-katta 0bc030c859 [SPARK-27246][SQL] Add an assert on invalid Scalar subquery plan with no column
## What changes were proposed in this pull request?

This PR proposes to add an assert on `ScalarSubquery`'s `dataType` because there's a possibility that `dataType` can be called alone before throwing analysis exception.

This was found while working on [SPARK-27088](https://issues.apache.org/jira/browse/SPARK-27088). This change calls `treeString` for logging purpose, and the specific test "scalar subquery with no column" under `AnalysisErrorSuite` was being failed with:

```
Caused by: sbt.ForkMain$ForkError: java.util.NoSuchElementException: next on empty iterator
	...
	at scala.collection.mutable.ArrayOps$ofRef.head(ArrayOps.scala:198)
	at org.apache.spark.sql.catalyst.expressions.ScalarSubquery.dataType(subquery.scala:251)
	at org.apache.spark.sql.catalyst.expressions.Alias.dataType(namedExpressions.scala:163)
        ...
	at org.apache.spark.sql.catalyst.trees.TreeNode.simpleString(TreeNode.scala:465)
        ...
	at org.apache.spark.sql.catalyst.rules.RuleExecutor$PlanChangeLogger.logRule(RuleExecutor.scala:176)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:116)
	...
```

The reason is that `treeString` for logging happened to call `dataType` on `ScalarSubquery` but one test has empty column plan. So, it happened to throw `NoSuchElementException` before checking analysis.

## How was this patch tested?

Manually tested.

```scala
ScalarSubquery(LocalRelation()).treeString
```

```
An exception or error caused a run to abort: assertion failed: Scala subquery should have only one column
java.lang.AssertionError: assertion failed: Scala subquery should have only one column
	at scala.Predef$.assert(Predef.scala:223)
	at org.apache.spark.sql.catalyst.expressions.ScalarSubquery.dataType(subquery.scala:252)
	at org.apache.spark.sql.catalyst.analysis.AnalysisErrorSuite.<init>(AnalysisErrorSuite.scala:116)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at java.lang.Class.newInstance(Class.java:442)
	at org.scalatest.tools.Runner$.genSuiteConfig(Runner.scala:1428)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$8(Runner.scala:1236)
	at scala.collection.immutable.List.map(List.scala:286)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1235)
```

Closes #24182 from sandeep-katta/subqueryissue.

Authored-by: sandeep-katta <sandeep.katta2007@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-26 09:25:57 +09:00
Ajith b61dce23d2 [SPARK-26961][CORE] Enable parallel classloading capability
## What changes were proposed in this pull request?

As per https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html
``Class loaders that support concurrent loading of classes are known as parallel capable class loaders and are required to register themselves at their class initialization time by invoking the ClassLoader.registerAsParallelCapable method. Note that the ClassLoader class is registered as parallel capable by default. However, its subclasses still need to register themselves if they are parallel capable. ``
 i.e we can have finer class loading locks by registering classloaders as parallel capable.  (Refer to deadlock due to macro lock  https://issues.apache.org/jira/browse/SPARK-26961).
All the classloaders we have are wrapper of URLClassLoader which by itself is parallel capable.
But this cannot be achieved by scala code due to static registration Refer https://github.com/scala/bug/issues/11429

## How was this patch tested?

All Existing UT must pass

Closes #24126 from ajithme/driverlock.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 19:07:30 -05:00
Liang-Chi Hsieh 8433ff6607 [SPARK-26847][SQL] Pruning nested serializers from object serializers: MapType support
## What changes were proposed in this pull request?

In SPARK-26837, we prune nested fields from object serializers if they are unnecessary in the query execution. SPARK-26837 leaves the support of MapType as a TODO item. This proposes to support map type.

## How was this patch tested?

Added tests.

Closes #24158 from viirya/SPARK-26847.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-25 15:36:58 -07:00
Liang-Chi Hsieh 5a36cf66ed [SPARK-27268][SQL] Add map_keys and map_values support in nested schema pruning.
## What changes were proposed in this pull request?

We need to add `map_keys` and `map_values` into `ProjectionOverSchema` to support those methods in nested schema pruning. This also adds end-to-end tests to SchemaPruningSuite.

## How was this patch tested?

Added tests.

Closes #24202 from viirya/SPARK-27268.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-25 15:32:01 -07:00
liuxian e4b36df2c0 [SPARK-27256][CORE][SQL] If the configuration is used to set the number of bytes, we'd better use bytesConf'.
## What changes were proposed in this pull request?
Currently, if we want to configure `spark.sql.files.maxPartitionBytes` to 256 megabytes, we must set  `spark.sql.files.maxPartitionBytes=268435456`, which is very unfriendly to users.

And if we set it like this:`spark.sql.files.maxPartitionBytes=256M`, we will  encounter this exception:
```
Exception in thread "main" java.lang.IllegalArgumentException:
 spark.sql.files.maxPartitionBytes should be long, but was 256M
        at org.apache.spark.internal.config.ConfigHelpers$.toNumber(ConfigBuilder.scala)
```
This PR use `bytesConf` to replace `longConf` or `intConf`,  if the configuration is used to set the number of bytes.
Configuration change list:
`spark.files.maxPartitionBytes`
`spark.files.openCostInBytes`
`spark.shuffle.sort.initialBufferSize`
`spark.shuffle.spill.initialMemoryThreshold`
`spark.sql.autoBroadcastJoinThreshold`
`spark.sql.files.maxPartitionBytes`
`spark.sql.files.openCostInBytes`
`spark.sql.defaultSizeInBytes`
## How was this patch tested?
1.Existing unit tests
2.Manual testing

Closes #24187 from 10110346/bytesConf.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-25 14:47:40 -07:00
Sean Owen 8bc304f97e [SPARK-26132][BUILD][CORE] Remove support for Scala 2.11 in Spark 3.0.0
## What changes were proposed in this pull request?

Remove Scala 2.11 support in build files and docs, and in various parts of code that accommodated 2.11. See some targeted comments below.

## How was this patch tested?

Existing tests.

Closes #23098 from srowen/SPARK-26132.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 10:46:42 -05:00
Takeshi Yamamuro b8a0f981f2 [SPARK-25196][SQL][FOLLOWUP] Fix wrong tests in StatisticsCollectionSuite
## What changes were proposed in this pull request?
This is a follow-up of #24047 and it fixed wrong tests in `StatisticsCollectionSuite`.

## How was this patch tested?
Pass Jenkins.

Closes #24198 from maropu/SPARK-25196-FOLLOWUP-2.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-25 21:02:01 +09:00
Maxim Gekk 52671d631d [SPARK-27008][SQL][FOLLOWUP] Fix typo from *_EANBLED to *_ENABLED
## What changes were proposed in this pull request?

This fixes a typo in the SQL config value: DATETIME_JAVA8API_**EANBLED** -> DATETIME_JAVA8API_**ENABLED**.

## How was this patch tested?

This was tested by `RowEncoderSuite` and `LiteralExpressionSuite`.

Closes #24194 from MaxGekk/date-localdate-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-24 17:16:33 -07:00
pgandhi a6c207c9c0 [SPARK-24935][SQL] fix Hive UDAF with two aggregation buffers
## What changes were proposed in this pull request?

Hive UDAF knows the aggregation mode when creating the aggregation buffer, so that it can create different buffers for different inputs: the original data or the aggregation buffer. Please see an example in the [sketches library](7f9e76e9e0/src/main/java/com/yahoo/sketches/hive/cpc/DataToSketchUDAF.java (L107)).

However, the Hive UDAF adapter in Spark always creates the buffer with partial1 mode, which can only deal with one input: the original data. This PR fixes it.

All credits go to pgandhi999 , who investigate the problem and study the Hive UDAF behaviors, and write the tests.

close https://github.com/apache/spark/pull/23778

## How was this patch tested?

a new test

Closes #24144 from cloud-fan/hive.

Lead-authored-by: pgandhi <pgandhi@verizonmedia.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-24 16:07:35 -07:00
Liang-Chi Hsieh 6f18ac9e99 [SPARK-27241][SQL] Support map_keys and map_values in SelectedField
## What changes were proposed in this pull request?

`SelectedField` doesn't support map_keys and map_values for now. When map key or value is complex struct, we should be able to prune unnecessary fields from keys/values. This proposes to add map_keys and map_values support to `SelectedField`.

## How was this patch tested?

Added tests.

Closes #24179 from viirya/SPARK-27241.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-23 23:13:31 -07:00
Takeshi Yamamuro 01e63053df [SPARK-25196][SPARK-27251][SQL][FOLLOWUP] Add synchronized for InMemoryRelation.statsOfPlanToCache
## What changes were proposed in this pull request?
This is a follow-up of #24047; to follow the `CacheManager.cachedData` lock semantics, this pr wrapped the `statsOfPlanToCache` update with `synchronized`.

## How was this patch tested?
Pass Jenkins

Closes #24178 from maropu/SPARK-24047-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-23 22:54:27 -07:00
Gengliang Wang 624288556d [SPARK-27085][SQL] Migrate CSV to File Data Source V2
## What changes were proposed in this pull request?

Migrate CSV to File Data Source V2.

## How was this patch tested?

Unit test

Closes #24005 from gengliangwang/CSVDataSourceV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-23 15:43:46 -07:00
Maxim Gekk 027ed2d11b [SPARK-23643][CORE][SQL][ML] Shrinking the buffer in hashSeed up to size of the seed parameter
## What changes were proposed in this pull request?

The hashSeed method allocates 64 bytes instead of 8. Other bytes are always zeros (thanks to default behavior of ByteBuffer). And they could be excluded from hash calculation because they don't differentiate inputs.

## How was this patch tested?

By running the existing tests - XORShiftRandomSuite

Closes #20793 from MaxGekk/hash-buff-size.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-23 11:26:09 -05:00
Marco Gaido fe317dc74e [SPARK-27243][SQL] RuleExecutor.dumpTimeSpent should not throw exception when empty
## What changes were proposed in this pull request?

`RuleExecutor.dumpTimeSpent` currently throws an exception when invoked before any rule is run or immediately after `RuleExecutor.reset`. The PR makes it returning an empty summary, which is the expected output instead.

## How was this patch tested?

added UT

Closes #24180 from mgaido91/SPARK-27243.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-23 09:49:20 +09:00
Ryan Blue 34e3cc7060 [SPARK-27108][SQL] Add parsed SQL plans for create, CTAS.
## What changes were proposed in this pull request?

This moves parsing `CREATE TABLE ... USING` statements into catalyst. Catalyst produces logical plans with the parsed information and those plans are converted to v1 `DataSource` plans in `DataSourceAnalysis`.

This prepares for adding v2 create plans that should receive the information parsed from SQL without being translated to v1 plans first.

This also makes it possible to parse in catalyst instead of breaking the parser across the abstract `AstBuilder` in catalyst and `SparkSqlParser` in core.

For more information, see the [mailing list thread](https://lists.apache.org/thread.html/54f4e1929ceb9a2b0cac7cb058000feb8de5d6c667b2e0950804c613%3Cdev.spark.apache.org%3E).

## How was this patch tested?

This uses existing tests to catch regressions. This introduces no behavior changes.

Closes #24029 from rdblue/SPARK-27108-add-parsed-create-logical-plans.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-22 13:58:54 -07:00
Jungtaek Lim (HeartSaVioR) 78d546fe15
[SPARK-27210][SS] Cleanup incomplete output files in ManifestFileCommitProtocol if task is aborted
## What changes were proposed in this pull request?

This patch proposes ManifestFileCommitProtocol to clean up incomplete output files in task level if task aborts. Please note that this works as 'best-effort', not kind of guarantee, as we have in HadoopMapReduceCommitProtocol.

## How was this patch tested?

Added UT.

Closes #24154 from HeartSaVioR/SPARK-27210.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-03-22 11:26:53 -07:00
Martin Junghanns 8efc5ec72e [SPARK-27174][SQL] Add support for casting integer types to binary
Co-authored-by: Philip Stutz <philip.stutzgmail.com>

## What changes were proposed in this pull request?

This PR adds support for casting

* `ByteType`
* `ShortType`
* `IntegerType`
* `LongType`

to `BinaryType`.

## How was this patch tested?

We added unit tests for casting instances of the above types. For validation, we used Javas `DataOutputStream` to compare the resulting byte array with the result of `Cast`.

We state that the contribution is our original work and that we license the work to the project under the project’s open source license.

cloud-fan we'd appreciate a review if you find the time, thx

Closes #24107 from s1ck/cast_to_binary.

Authored-by: Martin Junghanns <martin.junghanns@neotechnology.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-22 10:09:35 -07:00
Maxim Gekk a529be2930 [SPARK-27212][SQL] Eliminate TimeZone to ZoneId conversion in stringToTimestamp
## What changes were proposed in this pull request?

In the PR, I propose to avoid the `TimeZone` to `ZoneId` conversion in `DateTimeUtils.stringToTimestamp` by changing signature of the method, and require a parameter of `ZoneId` type. This will allow to avoid unnecessary conversion (`TimeZone` -> `String` -> `ZoneId`) per each row.

Also the PR avoids creation of `ZoneId` instances from `ZoneOffset` because `ZoneOffset` is a sub-class, and the conversion is unnecessary too.

## How was this patch tested?

It was tested by `DateTimeUtilsSuite` and `CastSuite`.

Closes #24155 from MaxGekk/stringtotimestamp-zoneid.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-22 18:01:29 +09:00
maryannxue 9f58d3b436 [SPARK-27236][TEST] Refactor log-appender pattern in tests
## What changes were proposed in this pull request?

Refactored code in tests regarding the "withLogAppender()" pattern by creating a general helper method in SparkFunSuite.

## How was this patch tested?

Passed existing tests.

Closes #24172 from maryannxue/log-appender.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-21 19:18:30 -07:00
John Zhuge 80565ce253 [SPARK-26946][SQL] Identifiers for multi-catalog
## What changes were proposed in this pull request?

- Support N-part identifier in SQL
- N-part identifier extractor in Analyzer

## How was this patch tested?

- A new unit test suite ResolveMultipartRelationSuite
- CatalogLoadingSuite

rblue cloud-fan mccheah

Closes #23848 from jzhuge/SPARK-26946.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-21 18:04:50 -07:00
Maxim Gekk 0f4f8160e6 [SPARK-27222][SQL] Support Instant and LocalDate in Literal.apply
## What changes were proposed in this pull request?

In the PR, I propose to extend `Literal.apply` to support constructing literals of `TimestampType` and `DateType` from `java.time.Instant` and `java.time.LocalDate`. The java classes have been already supported as external types for `TimestampType` and `DateType` by the PRs #23811  and #23913.

## How was this patch tested?

Added new tests to `LiteralExpressionSuite`.

Closes #24161 from MaxGekk/literal-instant-localdate.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-21 12:57:32 -07:00
Takeshi Yamamuro 0627850b7e [SPARK-25196][SQL] Extends the analyze column command for cached tables
## What changes were proposed in this pull request?
This pr extended `ANALYZE` commands to analyze column stats for cached table.

In common use cases, users read catalog table data, join/aggregate them, and then cache the result for following reuse. Since we are only allowed to analyze column statistics in catalog tables via ANALYZE commands, the current optimization depends on non-existing or inaccurate column statistics of cached data. So, it would be great if we could analyze cached data as follows;

```scala
scala> def printColumnStats(tableName: String) = {
     |   spark.table(tableName).queryExecution.optimizedPlan.stats.attributeStats.foreach {
     |     case (k, v) => println(s"[$k]: $v")
     |   }
     | }

scala> sql("SET spark.sql.cbo.enabled=true")
scala> sql("SET spark.sql.statistics.histogram.enabled=true")

scala> spark.range(1000).selectExpr("id % 33 AS c0", "rand() AS c1", "0 AS c2").write.saveAsTable("t")
scala> sql("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS c0, c1, c2")
scala> spark.table("t").groupBy("c0").agg(count("c1").as("v1"), sum("c2").as("v2")).createTempView("temp")

// Prints column statistics in catalog table `t`
scala> printColumnStats("t")
[c0#7073L]: ColumnStat(Some(33),Some(0),Some(32),Some(0),Some(8),Some(8),Some(Histogram(3.937007874015748,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;9f7c1c)),2)
[c1#7074]: ColumnStat(Some(944),Some(3.2108484832404915E-4),Some(0.997584797423909),Some(0),Some(8),Some(8),Some(Histogram(3.937007874015748,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;60a386b1)),2)
[c2#7075]: ColumnStat(Some(1),Some(0),Some(0),Some(0),Some(4),Some(4),Some(Histogram(3.937007874015748,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;5ffd29e8)),2)

// Prints column statistics on cached table `temp`
scala> sql("CACHE TABLE temp")
scala> printColumnStats("temp")
<No Column Statistics>

// Analyzes columns `v1` and `v2` on cached table `temp`
scala> sql("ANALYZE TABLE temp COMPUTE STATISTICS FOR COLUMNS v1, v2")

// Then, prints again
scala> printColumnStats("temp")
[v1#7084L]: ColumnStat(Some(2),Some(30),Some(31),Some(0),Some(8),Some(8),Some(Histogram(0.12992125984251968,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;49f7bb6f)),2)
[v2#7086L]: ColumnStat(Some(1),Some(0),Some(0),Some(0),Some(8),Some(8),Some(Histogram(0.12992125984251968,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;12701677)),2)

// Analyzes one left column and prints again
scala> sql("ANALYZE TABLE temp COMPUTE STATISTICS FOR COLUMNS c0")
scala> printColumnStats("temp")
[v1#7084L]: ColumnStat(Some(2),Some(30),Some(31),Some(0),Some(8),Some(8),Some(Histogram(0.12992125984251968,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;49f7bb6f)),2)
[v2#7086L]: ColumnStat(Some(1),Some(0),Some(0),Some(0),Some(8),Some(8),Some(Histogram(0.12992125984251968,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;12701677)),2)
[c0#7073L]: ColumnStat(Some(33),Some(0),Some(32),Some(0),Some(8),Some(8),Some(Histogram(0.12992125984251968,[Lorg.apache.spark.sql.catalyst.plans.logical.HistogramBin;1f5c1b81)),2)
```

## How was this patch tested?
Added tests in `CachedTableSuite` and `StatisticsCollectionSuite`.

Closes #24047 from maropu/SPARK-25196-4.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-21 09:20:35 -07:00
gengjiaan 22c9ed6a9c [MINOR][SQL] Put the grammar of database together, because this is good for maintenance and readability.
## What changes were proposed in this pull request?

The SQL grammar `SHOW DATABASES` is mixed in some grammar of table. I think should arrange the grammar of database together.
This is good for maintenance and readability.

## How was this patch tested?

No UT

Closes #24138 from beliefer/arrange-sql-grammar.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-21 06:41:04 -05:00
Bryan Cutler be08b415da [SPARK-27163][PYTHON] Cleanup and consolidate Pandas UDF functionality
## What changes were proposed in this pull request?

This change is a cleanup and consolidation of 3 areas related to Pandas UDFs:

1) `ArrowStreamPandasSerializer` now inherits from `ArrowStreamSerializer` and uses the base class `dump_stream`, `load_stream` to create Arrow reader/writer and send Arrow record batches.  `ArrowStreamPandasSerializer` makes the conversions to/from Pandas and converts to Arrow record batch iterators. This change removed duplicated creation of Arrow readers/writers.

2) `createDataFrame` with Arrow now uses `ArrowStreamPandasSerializer` instead of doing its own conversions from Pandas to Arrow and sending record batches through `ArrowStreamSerializer`.

3) Grouped Map UDFs now reuse existing logic in `ArrowStreamPandasSerializer` to send Pandas DataFrame results as a `StructType` instead of separating each column from the DataFrame. This makes the code a little more consistent with the Python worker, but does require that the returned StructType column is flattened out in `FlatMapGroupsInPandasExec` in Scala.

## How was this patch tested?

Existing tests and ran tests with pyarrow 0.12.0

Closes #24095 from BryanCutler/arrow-refactor-cleanup-UDFs.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-21 17:44:51 +09:00
Venkata krishnan Sowrirajan b1857a4d7d [SPARK-26894][SQL] Handle Alias as well in AggregateEstimation to propagate child stats
## What changes were proposed in this pull request?

Currently aliases are not handled in the Aggregate Estimation due to which stats are not getting propagated. This causes CBO join-reordering to not give optimal join plans. ProjectEstimation is already taking care of aliases, we need same logic for AggregateEstimation as well to properly propagate stats when CBO is enabled.

## How was this patch tested?

This patch is manually tested using the query Q83 of TPCDS benchmark (scale 1000)

Closes #23803 from venkata91/aggstats.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@qubole.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-21 11:21:56 +09:00
Shixiong Zhu c26379b446 [SPARK-27221][SQL] Improve the assert error message in TreeNode.parseToJson
## What changes were proposed in this pull request?

When `TreeNode.parseToJson` may throw an assert error without any error message when a TreeNode is not implemented properly, and it's hard to find the bad TreeNode implementation.

This PR adds the assert message to improve the error, like what `TreeNode.jsonFields` does.

## How was this patch tested?

Jenkins

Closes #24159 from zsxwing/SPARK-27221.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-21 11:15:05 +09:00
maryannxue 2e090ba628 [SPARK-27223][SQL] Remove private methods that skip conversion when passing user schemas for constructing a DataFrame
## What changes were proposed in this pull request?

When passing in a user schema to create a DataFrame, there might be mismatched nullability between the user schema and the the actual data. All related public interfaces now perform catalyst conversion using the user provided schema, which catches such mismatches to avoid runtime errors later on. However, there're private methods which allow this conversion to be skipped, so we need to remove these private methods which may lead to confusion and potential issues.

## How was this patch tested?

Passed existing tests. No new tests were added since this PR removed the private interfaces that would potentially cause null problems and other interfaces are covered already by existing tests.

Closes #24162 from maryannxue/spark-27223.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-21 11:13:25 +09:00
wangguangxin.cn 46f9f44918 [SPARK-27202][MINOR][SQL] Update comments to keep according with code
## What changes were proposed in this pull request?

Update comments in `InMemoryFileIndex.listLeafFiles` to keep according with code.

## How was this patch tested?

existing test cases

Closes #24146 from WangGuangxin/SPARK-27202.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-20 17:54:28 -05:00
Sean Owen c65f9b2bc3 [SPARK-26839][SQL] Work around classloader changes in Java 9 for Hive isolation
Note, this doesn't really resolve the JIRA, but makes the changes we can make so far that would be required to solve it.

## What changes were proposed in this pull request?

Java 9+ changed how ClassLoaders work. The two most salient points:
- The boot classloader no longer 'sees' the platform classes. A new 'platform classloader' does and should be the parent of new ClassLoaders
- The system classloader is no longer a URLClassLoader, so we can't get the URLs of JARs in its classpath

## How was this patch tested?

We'll see whether Java 8 tests still pass here. Java 11 tests do not fully pass at this point; more notes below. This does make progress on the failures though.

(NB: to test with Java 11, you need to build with Java 8 first, setting JAVA_HOME and java's executable correctly, then switch both to Java 11 for testing.)

Closes #24057 from srowen/SPARK-26839.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-20 09:12:52 -05:00
Maxim Gekk 1882912cca [SPARK-27199][SQL] Replace TimeZone by ZoneId in TimestampFormatter API
## What changes were proposed in this pull request?

In the PR, I propose to use `ZoneId` instead of `TimeZone` in:
- the `apply` and `getFractionFormatter ` methods of the `TimestampFormatter` object,
- and in implementations of the `TimestampFormatter` trait like `FractionTimestampFormatter`.

The reason of the changes is to avoid unnecessary conversion from `TimeZone` to `ZoneId` because `ZoneId` is used in `TimestampFormatter` implementations internally, and the conversion is performed via `String` which is not for free. Also taking into account that `TimeZone` instances are converted from `String` in some cases, the worse case looks like `String` -> `TimeZone` -> `String` -> `ZoneId`. The PR eliminates the unneeded conversions.

## How was this patch tested?

It was tested by `DateExpressionsSuite`, `DateTimeUtilsSuite` and `TimestampFormatterSuite`.

Closes #24141 from MaxGekk/zone-id.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 21:28:11 +09:00
Huon Wilson b67d369572 [SPARK-27099][SQL] Add 'xxhash64' for hashing arbitrary columns to Long
## What changes were proposed in this pull request?

This introduces a new SQL function 'xxhash64' for getting a 64-bit hash of an arbitrary number of columns.

This is designed to exactly mimic the 32-bit `hash`, which uses
MurmurHash3. The name is designed to be more future-proof than the
'hash', by indicating the exact algorithm used, similar to md5 and the
sha hashes.

## How was this patch tested?

The tests for the existing `hash` function were duplicated to run with `xxhash64`.

Closes #24019 from huonw/hash64.

Authored-by: Huon Wilson <Huon.Wilson@data61.csiro.au>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-20 16:34:34 +08:00
Darcy Shen 9a43852f17 [SPARK-27160][SQL] Fix DecimalType when building orc filters
## What changes were proposed in this pull request?
DecimalType Literal should not be casted to Long.

eg. For `df.filter("x < 3.14")`, assuming df (x in DecimalType) reads from a ORC table and uses the native ORC reader with predicate push down enabled, we will push down the `x < 3.14` predicate to the ORC reader via a SearchArgument.

OrcFilters will construct the SearchArgument, but not handle the DecimalType correctly.

The previous impl will construct `x < 3` from `x < 3.14`.

## How was this patch tested?
```
$ sbt
> sql/testOnly *OrcFilterSuite
> sql/testOnly *OrcQuerySuite -- -z "27160"
```

Closes #24092 from sadhen/spark27160.

Authored-by: Darcy Shen <sadhen@zoho.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-19 20:28:46 -07:00
Dongjoon Hyun 257391497b [SPARK-26975][SQL] Support nested-column pruning over limit/sample/repartition
## What changes were proposed in this pull request?

As [SPARK-26958](https://github.com/apache/spark/pull/23862/files) benchmark shows, nested-column pruning has limitations. This PR aims to remove the limitations on `limit/repartition/sample`. Here, repartition means `Repartition`, not `RepartitionByExpression`.

**PREPARATION**
```scala
scala> spark.range(100).map(x => (x, (x, s"$x" * 100))).toDF("col1", "col2").write.mode("overwrite").save("/tmp/p")
scala> sql("set spark.sql.optimizer.nestedSchemaPruning.enabled=true")
scala> spark.read.parquet("/tmp/p").createOrReplaceTempView("t")
```

**BEFORE**
```scala
scala> sql("SELECT col2._1 FROM (SELECT col2 FROM t LIMIT 1000000)").explain
== Physical Plan ==
CollectLimit 1000000
+- *(1) Project [col2#22._1 AS _1#28L]
   +- *(1) FileScan parquet [col2#22] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint>>

scala> sql("SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t)").explain
== Physical Plan ==
*(2) Project [col2#22._1 AS _1#33L]
+- Exchange RoundRobinPartitioning(1)
   +- *(1) Project [col2#22]
      +- *(1) FileScan parquet [col2#22] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint,_2:string>>
```

**AFTER**
```scala
scala> sql("SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t)").explain
== Physical Plan ==
Exchange RoundRobinPartitioning(1)
+- *(1) Project [col2#5._1 AS _1#11L]
   +- *(1) FileScan parquet [col2#5] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint>>
```

This supercedes https://github.com/apache/spark/pull/23542 and https://github.com/apache/spark/pull/23873 .

## How was this patch tested?

Pass the Jenkins with a newly added test suite.

Closes #23964 from dongjoon-hyun/SPARK-26975-ALIAS.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: DB Tsai <d_tsai@apple.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-19 20:24:22 -07:00
Dongjoon Hyun 4d5247778a [SPARK-27197][SQL][TEST] Add ReadNestedSchemaTest for file-based data sources
## What changes were proposed in this pull request?

The reader schema is said to be evolved (or projected) when it changed after the data is written by writers. Apache Spark file-based data sources have a test coverage for that; e.g. [ReadSchemaSuite.scala](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala). This PR aims to add a test coverage for nested columns by adding and hiding nested columns.

## How was this patch tested?

Pass the Jenkins with newly added tests.

Closes #24139 from dongjoon-hyun/SPARK-27197.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-20 00:22:05 +00:00
s71955 e402de5fd0 [SPARK-26176][SQL] Verify column names for CTAS with STORED AS
## What changes were proposed in this pull request?
Currently, users meet job abortions while creating a table using the Hive serde "STORED AS" with invalid column names. We had better prevent this by raising **AnalysisException** with a guide to use aliases instead like Paquet data source tables.
thus making compatible with error message shown while creating Parquet/ORC native table.

**BEFORE**
```scala
scala> sql("set spark.sql.hive.convertMetastoreParquet=false")
scala> sql("CREATE TABLE a STORED AS PARQUET AS SELECT 1 AS `COUNT(ID)`")
Caused by: java.lang.IllegalArgumentException: No enum constant parquet.schema.OriginalType.col1
```

**AFTER**
```scala
scala> sql("CREATE TABLE a STORED AS PARQUET AS SELECT 1 AS `COUNT(ID)`")
 Please use alias to rename it.;eption: Attribute name "count(ID)" contains invalid character(s) among " ,;{}()\n\t=".
```

## How was this patch tested?
Pass the Jenkins with the newly added test case.

Closes #24075 from sujith71955/master_serde.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 20:29:47 +08:00
Takeshi Yamamuro 901c7408a4 [SPARK-27161][SQL][FOLLOWUP] Drops non-keywords from docs/sql-keywords.md
## What changes were proposed in this pull request?
This pr is a follow-up of #24093 and includes fixes below;
 - Lists up all the keywords of Spark only (that is, drops non-keywords there); I listed up all the keywords of ANSI SQL-2011 in the previous commit (SPARK-26215).
 - Sorts the keywords in `SqlBase.g4` in a alphabetical order

## How was this patch tested?
Pass Jenkins.

Closes #24125 from maropu/SPARK-27161-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 20:18:40 +08:00
mwlon d5c08fcaab [SPARK-26555][SQL] make ScalaReflection subtype checking thread safe
## What changes were proposed in this pull request?

Make ScalaReflection subtype checking thread safe by adding a lock. There is a thread safety bug in the <:< operator in all versions of scala (https://github.com/scala/bug/issues/10766).

## How was this patch tested?

Existing tests and a new one for the new subtype checking function.

Closes #24085 from mwlon/SPARK-26555.

Authored-by: mwlon <mloncaric@hmc.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 18:22:01 +08:00
wuyi a8af23d7ab [SPARK-27193][SQL] CodeFormatter should format multiple comment lines correctly
## What changes were proposed in this pull request?

when enable `spark.sql.codegen.comments`,  there will be multiple comment lines. However, CodeFormatter can not handle multiple comment lines currently:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ /**
 * Codegend pipeline for stage (id=1)
 * *(1) Project [(id#0L + 1) AS (id + 1)#3L]
 * +- *(1) Filter (id#0L = 1)
 *    +- *(1) Range (0, 10, step=1, splits=4)
 */
/* 006 */ // codegenStageId=1
/* 007 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
```

After applying this pr:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ /**
/* 006 */  * Codegend pipeline for stage (id=1)
/* 007 */  * *(1) Project [(id#0L + 1) AS (id + 1)#4L]
/* 008 */  * +- *(1) Filter (id#0L = 1)
/* 009 */  *    +- *(1) Range (0, 10, step=1, splits=2)
/* 010 */  */
/* 011 */ // codegenStageId=1
/* 012 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
```

## How was this patch tested?

Tested Manually.

Closes #24133 from Ngone51/fix-codeformatter-for-multi-comment-lines.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 14:47:51 +08:00
Gengliang Wang 28d35c8578 [SPARK-27162][SQL] Add new method asCaseSensitiveMap in CaseInsensitiveStringMap
## What changes were proposed in this pull request?

Currently, DataFrameReader/DataFrameReader supports setting Hadoop configurations via method `.option()`.
E.g, the following test case should be passed in both ORC V1 and V2
```
  class TestFileFilter extends PathFilter {
    override def accept(path: Path): Boolean = path.getParent.getName != "p=2"
  }

  withTempPath { dir =>
      val path = dir.getCanonicalPath

      val df = spark.range(2)
      df.write.orc(path + "/p=1")
      df.write.orc(path + "/p=2")
      val extraOptions = Map(
        "mapred.input.pathFilter.class" -> classOf[TestFileFilter].getName,
        "mapreduce.input.pathFilter.class" -> classOf[TestFileFilter].getName
      )
      assert(spark.read.options(extraOptions).orc(path).count() === 2)
    }
  }
```
While Hadoop Configurations are case sensitive, the current data source V2 APIs are using `CaseInsensitiveStringMap` in the top level entry `TableProvider`.
To create Hadoop configurations correctly, I suggest
1. adding a new method `asCaseSensitiveMap` in `CaseInsensitiveStringMap`.
2. Make `CaseInsensitiveStringMap` read-only to ambiguous conversion in `asCaseSensitiveMap`

## How was this patch tested?

Unit test

Closes #24094 from gengliangwang/originalMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 13:35:47 +08:00
Dongjoon Hyun 26e9849cb4 [SPARK-27195][SQL][TEST] Add AvroReadSchemaSuite
## What changes were proposed in this pull request?

The reader schema is said to be evolved (or projected) when it changed after the data is written by writers. Apache Spark file-based data sources have a test coverage for that, [ReadSchemaSuite.scala](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala). This PR aims to add `AvroReadSchemaSuite` to ensure the minimal consistency among file-based data sources and prevent a future regression in Avro data source.

## How was this patch tested?

Pass the Jenkins with the newly added test suite.

Closes #24135 from dongjoon-hyun/SPARK-27195.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-18 20:10:30 -07:00
Ryan Blue e348f14259 [SPARK-26811][SQL] Add capabilities to v2.Table
## What changes were proposed in this pull request?

This adds a new method, `capabilities` to `v2.Table` that returns a set of `TableCapability`. Capabilities are used to fail queries during analysis checks, `V2WriteSupportCheck`, when the table does not support operations, like truncation.

## How was this patch tested?

Existing tests for regressions, added new analysis suite, `V2WriteSupportCheckSuite`, for new capability checks.

Closes #24012 from rdblue/SPARK-26811-add-capabilities.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-18 18:25:11 +08:00
Wenchen Fan dbcb4792f2 [SPARK-27161][SQL] improve the document of SQL keywords
## What changes were proposed in this pull request?

Make it more clear about how Spark categories keywords regarding to the config `spark.sql.parser.ansi.enabled`

## How was this patch tested?

existing tests

Closes #24093 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-18 15:19:52 +09:00
Jungtaek Lim (HeartSaVioR) 4adbcdc424 [SPARK-22000][SQL][FOLLOW-UP] Fix bad test to ensure it can test properly
## What changes were proposed in this pull request?

There was some mistake on test code: it has wrong assertion. The patch proposes fixing it, as well as fixing other stuff to make test really pass.

## How was this patch tested?

Fixed unit test.

Closes #24112 from HeartSaVioR/SPARK-22000-hotfix.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-17 08:25:40 +09:00
Dilip Biswal aea9a574c4 [SPARK-27134][SQL] array_distinct function does not work correctly with columns containing array of array
## What changes were proposed in this pull request?
Correct the logic to compute the distinct.

Below is a small repro snippet.

```
scala> val df = Seq(Seq(Seq(1, 2), Seq(1, 2), Seq(1, 2), Seq(3, 4), Seq(4, 5))).toDF("array_col")
df: org.apache.spark.sql.DataFrame = [array_col: array<array<int>>]

scala> val distinctDF = df.select(array_distinct(col("array_col")))
distinctDF: org.apache.spark.sql.DataFrame = [array_distinct(array_col): array<array<int>>]

scala> df.show(false)
+----------------------------------------+
|array_col                               |
+----------------------------------------+
|[[1, 2], [1, 2], [1, 2], [3, 4], [4, 5]]|
+----------------------------------------+
```
Error
```
scala> distinctDF.show(false)
+-------------------------+
|array_distinct(array_col)|
+-------------------------+
|[[1, 2], [1, 2], [1, 2]] |
+-------------------------+
```
Expected result
```
scala> distinctDF.show(false)
+-------------------------+
|array_distinct(array_col)|
+-------------------------+
|[[1, 2], [3, 4], [4, 5]] |
+-------------------------+
```
## How was this patch tested?
Added an additional test.

Closes #24073 from dilipbiswal/SPARK-27134.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 14:30:42 -05:00
Dilip Biswal 7a136f8670 [SPARK-27096][SQL][FOLLOWUP] Do the correct validation of join types in R side and fix join docs for scala, python and r
## What changes were proposed in this pull request?
This is a minor follow-up PR for SPARK-27096. The original PR reconciled the join types supported between dataset and sql interface. In case of R, we do the join type validation in the R side. In this PR we do the correct validation and adds tests in R to test all the join types along with the error condition. Along with this, i made the necessary doc correction.

## How was this patch tested?
Add R tests.

Closes #24087 from dilipbiswal/joinfix_followup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-16 13:04:54 +09:00
Zhu, Lipeng 8ee09f26d5 [SPARK-27159][SQL] update mssql server dialect to support binary type
## What changes were proposed in this pull request?

Change the binary type mapping from default blob to varbinary(max) for mssql server.
https://docs.microsoft.com/en-us/sql/t-sql/data-types/binary-and-varbinary-transact-sql?view=sql-server-2017
![image](https://user-images.githubusercontent.com/698621/54351715-0e8c8780-468b-11e9-8931-7ecb85c5ad6b.png)

## How was this patch tested?

Unit test.

Closes #24091 from lipzhu/SPARK-27159.

Authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-15 20:21:59 -05:00
Gengliang Wang 2a37d6ed93 [SPARK-27132][SQL] Improve file source V2 framework
## What changes were proposed in this pull request?

During the migration of CSV V2(https://github.com/apache/spark/pull/24005), I find that we can improve the file source v2 framework by:
1. check duplicated column names in both read and write
2. Not all the file sources support filter push down. So remove `SupportsPushDownFilters` from FileScanBuilder
3. The method `isSplitable` might require data source options. Add a new member `options` to FileScan.
4. Make `FileTable.schema` a lazy value instead of a method.

## How was this patch tested?

Unit test

Closes #24066 from gengliangwang/reviseFileSourceV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-15 11:58:03 +08:00
Dongjoon Hyun 74d2f04183 [SPARK-27166][SQL] Improve printSchema to print up to the given level
## What changes were proposed in this pull request?

This PR aims to improve `printSchema` to be able to print up to the given level of the schema.

```scala
scala> val df = Seq((1,(2,(3,4)))).toDF
df: org.apache.spark.sql.DataFrame = [_1: int, _2: struct<_1: int, _2: struct<_1: int, _2: int>>]

scala> df.printSchema
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)
| | |-- _1: integer (nullable = false)
| | |-- _2: integer (nullable = false)

scala> df.printSchema(1)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)

scala> df.printSchema(2)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)

scala> df.printSchema(3)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)
| | |-- _1: integer (nullable = false)
| | |-- _2: integer (nullable = false)
```

## How was this patch tested?

Pass the Jenkins with the newly added test case.

Closes #24098 from dongjoon-hyun/SPARK-27166.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 20:27:55 -07:00
Gengliang Wang 6d22ee3969 [SPARK-27136][SQL] Remove data source option check_files_exist
## What changes were proposed in this pull request?

The data source option check_files_exist is introduced in In #23383 when the file source V2 framework is implemented. In the PR, FileIndex was created as a member of FileTable, so that we could implement partition pruning like 0f9fcab in the future. At that time `FileIndex`es will always be created for file writes, so we needed the option to decide whether to check file existence.

After https://github.com/apache/spark/pull/23774, the option is not needed anymore, since Dataframe writes won't create unnecessary FileIndex. This PR is to remove the option.

## How was this patch tested?

Unit test.

Closes #24069 from gengliangwang/removeOptionCheckFilesExist.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-15 10:19:26 +08:00
Dave DeCaprio 8819eaba4d [SPARK-26917][SQL] Further reduce locks in CacheManager
## What changes were proposed in this pull request?

Further load increases in our production environment have shown that even the read locks can cause some contention, since they contain a mechanism that turns a read lock into an exclusive lock if a writer has been starved out.  This PR reduces the potential for lock contention even further than https://github.com/apache/spark/pull/23833.  Additionally, it uses more idiomatic scala than the previous implementation.

cloud-fan & gatorsmile This is a relatively minor improvement to the previous CacheManager changes.  At this point, I think we finally are doing the minimum possible amount of locking.

## How was this patch tested?

Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.

Closes #24028 from DaveDeCaprio/read-locks-master.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-15 10:13:34 +08:00
Shahid 8b5224097b [SPARK-27145][MINOR] Close store in the SQLAppStatusListenerSuite after test
## What changes were proposed in this pull request?
We create many stores in the SQLAppStatusListenerSuite, but we need to the close store after test.

## How was this patch tested?
Existing tests

Closes #24079 from shahidki31/SPARK-27145.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-14 13:08:41 -07:00
Yuming Wang da7db9abf6 [SPARK-23749][SQL] Replace built-in Hive API (isSub/toKryo) and remove OrcProto.Type usage
## What changes were proposed in this pull request?

In order to make the upgrade built-in Hive changes smaller.
This pr workaround the simplest 3 API changes first.

## How was this patch tested?

manual tests

Closes #24018 from wangyum/SPARK-23749.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 11:41:40 -07:00
Takeshi Yamamuro 66c5cd2d9c [SPARK-27151][SQL] ClearCacheCommand extends IgnoreCahedData to avoid plan node copys
## What changes were proposed in this pull request?
In SPARK-27011, we introduced `IgnoreCahedData` to avoid plan node copys in `CacheManager`.
Since `ClearCacheCommand` has no argument, it also can extend `IgnoreCahedData`.

## How was this patch tested?
Pass Jenkins.

Closes #24081 from maropu/SPARK-27011-2.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 11:36:16 -07:00
Takeshi Yamamuro bacffb8810 [SPARK-23264][SQL] Make INTERVAL keyword optional in INTERVAL clauses when ANSI mode enabled
## What changes were proposed in this pull request?
This pr updated parsing rules in `SqlBase.g4` to support a SQL query below when ANSI mode enabled;
```
SELECT CAST('2017-08-04' AS DATE) + 1 days;
```
The current master cannot parse it though, other dbms-like systems support the syntax (e.g., hive and mysql). Also, the syntax is frequently used in the official TPC-DS queries.

This pr added new tokens as follows;
```
YEAR | YEARS | MONTH | MONTHS | WEEK | WEEKS | DAY | DAYS | HOUR | HOURS | MINUTE
MINUTES | SECOND | SECONDS | MILLISECOND | MILLISECONDS | MICROSECOND | MICROSECONDS
```
Then, it registered the keywords below as the ANSI reserved (this follows SQL-2011);
```
 DAY | HOUR | MINUTE | MONTH | SECOND | YEAR
```

## How was this patch tested?
Added tests in `SQLQuerySuite`, `ExpressionParserSuite`, and `TableIdentifierParserSuite`.

Closes #20433 from maropu/SPARK-23264.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-14 10:45:29 +09:00
Dongjoon Hyun 250946ff93 [SPARK-27123][SQL][FOLLOWUP] Use isRenaming check for limit too.
## What changes were proposed in this pull request?

This is a followup for https://github.com/apache/spark/pull/24049 to reduce the scope of pattern based on the review comments.

## How was this patch tested?

Pass the existing test.

Closes #24082 from dongjoon-hyun/SPARK-27123-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-13 15:01:01 -07:00
Jungtaek Lim (HeartSaVioR) 733f2c0b98 [MINOR][SQL] Deduplicate huge if statements in get between specialized getters
## What changes were proposed in this pull request?

This patch deduplicates the huge if statements regarding getting value between specialized getters.

## How was this patch tested?

Existing UT.

Closes #24016 from HeartSaVioR/MINOR-deduplicate-get-from-specialized-getters.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-13 15:52:21 -05:00
Dongjoon Hyun 3221bf4cd5 [SPARK-27034][SPARK-27123][SQL][FOLLOWUP] Update Nested Schema Pruning BM result with EC2
## What changes were proposed in this pull request?

This is a follow up PR for #23943 in order to update the benchmark result with EC2 `r3.xlarge` instance.

## How was this patch tested?

N/A. (Manually compare the diff)

Closes #24078 from dongjoon-hyun/SPARK-27034.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-13 20:27:10 +00:00
Wenchen Fan 2a80a4cd39 [SPARK-27106][SQL] merge CaseInsensitiveStringMap and DataSourceOptions
## What changes were proposed in this pull request?

It's a little awkward to have 2 different classes(`CaseInsensitiveStringMap` and `DataSourceOptions`) to present the options in data source and catalog API.

This PR merges these 2 classes, while keeping the name `CaseInsensitiveStringMap`, which is more precise.

## How was this patch tested?

existing tests

Closes #24025 from cloud-fan/option.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-14 01:23:27 +08:00
Dave DeCaprio 812ad55461 [SPARK-26103][SQL] Limit the length of debug strings for query plans
## What changes were proposed in this pull request?

The PR puts in a limit on the size of a debug string generated for a tree node.  Helps to fix out of memory errors when large plans have huge debug strings.   In addition to SPARK-26103, this should also address SPARK-23904 and SPARK-25380.  AN alternative solution was proposed in #23076, but that solution doesn't address all the cases that can cause a large query.  This limit is only on calls treeString that don't pass a Writer, which makes it play nicely with #22429, #23018 and #23039.  Full plans can be written to files, but truncated plans will be used when strings are held in memory, such as for the UI.

- A new configuration parameter called spark.sql.debug.maxPlanLength was added to control the length of the plans.
- When plans are truncated, "..." is printed to indicate that it isn't a full plan
- A warning is printed out the first time a truncated plan is displayed. The warning explains what happened and how to adjust the limit.

## How was this patch tested?

Unit tests were created for the new SizeLimitedWriter.  Also a unit test for TreeNode was created that checks that a long plan is correctly truncated.

Closes #23169 from DaveDeCaprio/text-plan-size.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-13 09:58:43 -07:00
Wenchen Fan d3813d8b21 [SPARK-27064][SS] create StreamingWrite at the beginning of streaming execution
## What changes were proposed in this pull request?

According to the [design](https://docs.google.com/document/d/1vI26UEuDpVuOjWw4WPoH2T6y8WAekwtI7qoowhOFnI4/edit?usp=sharing), the life cycle of `StreamingWrite` should be the same as the read side `MicroBatch/ContinuousStream`, i.e. each run of the stream query, instead of each epoch.

This PR fixes it.

## How was this patch tested?

existing tests

Closes #23981 from cloud-fan/dsv2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-13 19:47:54 +08:00
Liang-Chi Hsieh f55c760df6 [SPARK-27034][SQL][FOLLOWUP] Rename ParquetSchemaPruning to SchemaPruning
## What changes were proposed in this pull request?

This is a followup to #23943. This proposes to rename ParquetSchemaPruning to SchemaPruning as ParquetSchemaPruning supports both Parquet and ORC v1 now.

## How was this patch tested?

Existing tests.

Closes #24077 from viirya/nested-schema-pruning-orc-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-13 20:12:01 +09:00
Jungtaek Lim (HeartSaVioR) 1b06cda532 [MINOR][SQL] Refactor RowEncoder to use existing (De)serializerBuildHelper methods
## What changes were proposed in this pull request?

This patch proposes to reuse existing methods in (De)serializerBuildHelper in RowEncoder to achieve deduplication as well as consistent creation of serialization/deserialization of same type.

## How was this patch tested?

Existing UT.

Closes #24014 from HeartSaVioR/SPARK-27092.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-13 10:54:47 +08:00
Takeshi Yamamuro 1e9469bb7a [SPARK-26976][SQL] Forbid reserved keywords as identifiers when ANSI mode is on
## What changes were proposed in this pull request?
This pr added code to forbid reserved keywords as identifiers when ANSI mode is on.
This is a follow-up of SPARK-26215(#23259).

## How was this patch tested?
Added tests in `TableIdentifierParserSuite`.

Closes #23880 from maropu/SPARK-26976.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-13 11:20:27 +09:00
Ajith e60d8fce0b [SPARK-27045][SQL] SQL tab in UI shows actual SQL instead of callsite in case of SparkSQLDriver
## What changes were proposed in this pull request?

When we run sql in spark via SparkSQLDriver (thrift server, spark-sql), SQL string is siet via ``setJobDescription``. the SparkUI SQL tab must show SQL instead of stacktrace in case ``setJobDescription`` is set which is more useful to end user. Instead it currently shows in description column the callsite shortform which is less useful

![image](https://user-images.githubusercontent.com/22072336/53734682-aaa7d900-3eaa-11e9-957b-0e5006db417e.png)

## How was this patch tested?

Manually:
![image](https://user-images.githubusercontent.com/22072336/53734657-9f54ad80-3eaa-11e9-8dc5-2b38f6970f4e.png)

Closes #23958 from ajithme/sqlui.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 16:14:29 -07:00
Liang-Chi Hsieh b0c2b3bfd9 [SPARK-27034][SQL] Nested schema pruning for ORC
## What changes were proposed in this pull request?

We only supported nested schema pruning for Parquet previously. This proposes to support nested schema pruning for ORC too.

Note: This only covers ORC v1. For ORC v2, the necessary change is at the schema pruning rule. We should deal with ORC v2 as a TODO item, in order to reduce review burden.

## How was this patch tested?

Added tests.

Closes #23943 from viirya/nested-schema-pruning-orc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 15:39:16 -07:00
Dongjoon Hyun 78314af580 [SPARK-27123][SQL] Improve CollapseProject to handle projects cross limit/repartition/sample
## What changes were proposed in this pull request?

`CollapseProject` optimizer rule simplifies some plans by merging the adjacent projects and performing alias substitutions.
```scala
scala> sql("SELECT b c FROM (SELECT a b FROM t)").explain
== Physical Plan ==
*(1) Project [a#5 AS c#1]
+- Scan hive default.t [a#5], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#5]
```

We can do that more complex cases like the following. This PR aims to handle adjacent projects across limit/repartition/sample. Here, repartition means `Repartition`, not `RepartitionByExpression`.

**BEFORE**
```scala
scala> sql("SELECT b c FROM (SELECT /*+ REPARTITION(1) */ a b FROM t)").explain
== Physical Plan ==
*(2) Project [b#0 AS c#1]
+- Exchange RoundRobinPartitioning(1)
   +- *(1) Project [a#5 AS b#0]
      +- Scan hive default.t [a#5], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#5]
```

**AFTER**
```scala
scala> sql("SELECT b c FROM (SELECT /*+ REPARTITION(1) */ a b FROM t)").explain
== Physical Plan ==
Exchange RoundRobinPartitioning(1)
+- *(1) Project [a#11 AS c#7]
   +- Scan hive default.t [a#11], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#11]
```

## How was this patch tested?

Pass the Jenkins with the newly added and updated test cases.

Closes #24049 from dongjoon-hyun/SPARK-27123.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-12 21:45:40 +00:00
zuotingbing 3f9247de1e [SPARK-27010][SQL] Find out the actual port number when hive.server2.thrift.port=0
## What changes were proposed in this pull request?
Currently, if we set hive.server2.thrift.port=0, it hard to find out the actual port number which one we should use when using beeline to connect.

before:
![2019-02-28_170942](https://user-images.githubusercontent.com/24823338/53557240-779ad800-3b80-11e9-9567-175f28aa61da.png)

after:
![2019-02-28_170904](https://user-images.githubusercontent.com/24823338/53557255-7f5a7c80-3b80-11e9-8ba6-9764c03e5407.png)

use beeline to connect success:
![2019-02-28_170844](https://user-images.githubusercontent.com/24823338/53557267-85e8f400-3b80-11e9-90a5-f7f53a51cc32.png)

## How was this patch tested?
 manual tests

Closes #23917 from zuotingbing/SPARK-27010.

Authored-by: zuotingbing <zuo.tingbing9@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:38:41 -05:00
shivusondur 4b6d39d85d [SPARK-27090][CORE] Removing old LEGACY_DRIVER_IDENTIFIER ("<driver>")
## What changes were proposed in this pull request?
LEGACY_DRIVER_IDENTIFIER and its reference are removed.
corresponding references test are updated.

## How was this patch tested?
tested  UT test cases

Closes #24026 from shivusondur/newjira2.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:29:39 -05:00
Shahid 1853db3186 [SPARK-27125][SQL][TEST] Add test suite for sql execution page
## What changes were proposed in this pull request?
Added test suite for AllExecutionsPage class. Checked the scenarios for SPARK-27019 and SPARK-27075.

## How was this patch tested?
Added UT, manually tested

Closes #24052 from shahidki31/SPARK-27125.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 10:15:28 -05:00
Ajith b8dd84b9e4 [SPARK-27011][SQL] reset command fails with cache
## What changes were proposed in this pull request?

When cache is enabled ( i.e once cache table command is executed), any following sql will trigger
 CacheManager#lookupCachedData which will create a copy of the tree node, which inturn calls TreeNode#makeCopy. Here the problem is it will try to create a copy instance. But as ResetCommand is a case object this will fail

## How was this patch tested?

Added UT to reproduce the issue

Closes #23918 from ajithme/reset.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-12 11:02:09 +08:00
Maxim Gekk 60be6d2ea3 [SPARK-27109][SQL] Refactoring of TimestampFormatter and DateFormatter
## What changes were proposed in this pull request?

In PR, I propose to refactor the `parse()` method of `Iso8601DateFormatter`/`Iso8601DateFormatter` and `toInstantWithZoneId` of `toInstantWithZoneId` to achieve the following:
- Avoid unnecessary conversion of parsed input to `java.time.Instant` before converting it to micros and days. Necessary information exists in `ZonedDateTime` already, and micros/days can be extracted from the former one.
- Avoid additional extraction of LocalTime from parsed object, more precisely, double query of `TemporalQueries.localTime` from `temporalAccessor`.
- Avoid additional extraction of zone id from parsed object, in particular, double query of `TemporalQueries.offset()`.
- Using `ZoneOffset.UTC` instead of `ZoneId.of` in `DateFormatter`. This allows to avoid looking for zone offset by zone id.

## How was this patch tested?

By existing test suite `DateTimeUtilsSuite`, `TimestampFormatterSuite` and `DateFormatterSuite`.

Closes #24030 from MaxGekk/query-localtime.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 19:02:30 -05:00
Hyukjin Kwon 3725b1324f [SPARK-26923][SQL][R] Refactor ArrowRRunner and RRunner to share one BaseRRunner
## What changes were proposed in this pull request?

This PR proposes to have one base R runner.

In the high level,

Previously, it had `ArrowRRunner` and it inherited `RRunner`:

```
└── RRunner
    └── ArrowRRunner
```

After this PR, now it has a `BaseRRunner`, and `ArrowRRunner` and `RRunner` inherit `BaseRRunner`:

```
└── BaseRRunner
    ├── ArrowRRunner
    └── RRunner
```

This way is consistent with Python's.

In more details, see below:

```scala
class BaseRRunner[IN, OUT] {

  def compute: Iterator[OUT] = {
    ...
    newWriterThread(...).start()
    ...
    newReaderIterator(...)
    ...
  }

  // Make a thread that writes data from JVM to R process
  abstract protected def newWriterThread(..., iter: Iterator[IN], ...): WriterThread

  // Make an iterator that reads data from the R process to JVM
  abstract protected def newReaderIterator(...): ReaderIterator

  abstract class WriterThread(..., iter: Iterator[IN], ...) extends Thread {
    override def run(): Unit {
      ...
      writeIteratorToStream(...)
      ...
    }

    // Actually writing logic to the socket stream.
    abstract protected def writeIteratorToStream(dataOut: DataOutputStream): Unit
  }

  abstract class ReaderIterator extends Iterator[OUT] {
    override def hasNext(): Boolean = {
      ...
      read(...)
      ...
    }

    override def next(): OUT = {
      ...
      hasNext()
      ...
    }

    // Actually reading logic from the socket stream.
    abstract protected def read(...): OUT
  }
}
```

```scala
case [Arrow]RRunner extends BaseRRunner {
  override def newWriterThread(...) {
    new WriterThread(...) {
      override def writeIteratorToStream(...) {
        ...
      }
    }
  }

  override def newReaderIterator(...) {
    new ReaderIterator(...) {
      override def read(...) {
        ...
      }
    }
  }
}
```

## How was this patch tested?

Manually tested and existing tests should cover.

Closes #23977 from HyukjinKwon/SPARK-26923.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-12 08:45:29 +09:00
Wenchen Fan 31878c9daa [SPARK-27119][SQL] Do not infer schema when reading Hive serde table with native data source
## What changes were proposed in this pull request?

In Spark 2.1, we hit a correctness bug. When reading a Hive serde parquet table with the native parquet data source, and the actual file schema doesn't match the table schema in Hive metastore(only upper/lower case difference), the query returns 0 results.

The reason is that, the parquet reader is case sensitive. If we push down filters with column names that don't match the file physical schema case-sensitively, no data will be returned.

To fix this bug, there were 2 solutions proposed at that time:
1. Add a config to optionally disable parquet filter pushdown, and make parquet column pruning case insensitive.
https://github.com/apache/spark/pull/16797

2. Infer the actual schema from data files, when reading Hive serde table with native data source. A config is provided to disable it.
https://github.com/apache/spark/pull/17229

Solution 2 was accepted and merged to Spark 2.1.1

In Spark 2.4, we refactored the parquet data source a little:
1. do parquet filter pushdown with the actual file schema.
https://github.com/apache/spark/pull/21696

2. make parquet filter pushdown case insensitive.
https://github.com/apache/spark/pull/22197

3. make parquet column pruning case insensitive.
https://github.com/apache/spark/pull/22148

With these patches, the correctness bug in Spark 2.1 no longer exists, and the schema inference becomes unnecessary.

To be safe, this PR just changes the default value to NEVER_INFER, so that users can set it back to INFER_AND_SAVE. If we don't receive any bug reports for it, we can remove the related code in the next release.

## How was this patch tested?

existing tests

Closes #24041 from cloud-fan/infer.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-11 09:44:29 -07:00
Jagadesh Kiran d9978fb4e4 [SPARK-26860][PYSPARK][SPARKR] Fix for RangeBetween and RowsBetween docs to be in sync with spark documentation
The docs describing RangeBetween & RowsBetween for pySpark & SparkR are not in sync with Spark description.

a. Edited PySpark and SparkR docs  and made description same for both RangeBetween and RowsBetween
b. created executable examples in both pySpark and SparkR documentation
c. Locally tested the patch for scala Style checks and UT for checking no testcase failures

Closes #23946 from jagadesh-kiran/master.

Authored-by: Jagadesh Kiran <jagadesh.n@in.verizon.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 08:53:09 -05:00
Wenchen Fan 8114b63d56 [SPARK-27117][SQL] current_date/current_timestamp should not refer to columns with ansi parser mode
## What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/19559 .

It revisits https://issues.apache.org/jira/browse/SPARK-27117 , which should be an invalid use case according to the SQL standard.

`current_date/current_timestamp` are reserved keywords, if users want to access columns named `current_date/current_timestamp`, they should quote the name like ```select `current_date` from tbl```

If ansi mode is not enabled(which is the default), this PR won't introduce any changes.

## How was this patch tested?

a new test case

Closes #24039 from cloud-fan/current_datetime.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 20:40:03 +08:00
Dilip Biswal 1b9fd67904 [SPARK-27096][SQL] Reconcile the join types between data frame and sql interface
## What changes were proposed in this pull request?
Currently in the grammar file, we have the joinType rule defined as following :
```
joinType
    : INNER?
   ....
   ....
    | LEFT SEMI
    | LEFT? ANTI
    ;
```
The keyword LEFT is optional for ANTI join even though its not optional for SEMI join. When
using data frame interface join type "anti" is not allowed. The allowed types are "left_anti" or
"leftanti" for anti joins. ~~In this PR, i am making the LEFT keyword mandatory for ANTI joins so
it aligns better with the LEFT SEMI join in the grammar file and also the join types allowed from dataframe api.~~

This PR makes LEFT optional for SEMI join in .g4 and add "semi" and "anti" join types from dataframe.

~~I have not opened any JIRA for this as probably we may need some discussion to see if
we are going to address this or not.~~

## How was this patch tested?
Modified the join type tests.

Closes #23982 from dilipbiswal/join_fix.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 14:02:21 +08:00
Takeshi Yamamuro 7a9537c338 [SPARK-21351][SQL] Remove the UpdateAttributeNullability rule from the optimizer
## What changes were proposed in this pull request?
This pr removed `UpdateAttributeNullability` from the optimizer because the same logic happens in the analyzer. See SPARK-26459(#23390) for more detailed discussion.

## How was this patch tested?
N/A

Closes #23508 from maropu/SPARK-21351.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 13:34:14 +08:00
Jungtaek Lim (HeartSaVioR) f0bde69ebc [MINOR][SQL] Throw better exception for Encoder with tuple more than 22 elements
## What changes were proposed in this pull request?

This patch proposes to throw better exception with better error message when encoding to tuple which elements are more than 22.

**BEFORE**
```scala
scala> import org.apache.spark.sql.catalyst.encoders._
scala> val encoders = (0 to 22).map(_ => org.apache.spark.sql.Encoders.scalaInt.asInstanceOf[ExpressionEncoder[_]])
scala> ExpressionEncoder.tuple(encoders)
java.lang.ClassNotFoundException: scala.Tuple23
```

**AFTER**
```scala
scala> ExpressionEncoder.tuple(encoders)
java.lang.UnsupportedOperationException: Due to Scala's limited support of tuple, tuple with more than 22 elements are not supported.
```

## How was this patch tested?

Added UT.

Closes #24046 from HeartSaVioR/MINOR-throw-better-exception-for-tuple-more-than-22.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-11 13:44:45 +09:00
Takeshi Yamamuro f0927d8ac4 [SPARK-27110][SQL] Moves some functions from AnalyzeColumnCommand to command/CommandUtils
## What changes were proposed in this pull request?
To reuse some common logics for improving `Analyze` commands (See the description of `SPARK-25196` for details), this pr moved some functions from `AnalyzeColumnCommand` to `command/CommandUtils`.  A follow-up pr will add code to extend `Analyze` commands for cached tables.

## How was this patch tested?
Existing tests.

Closes #22204 from maropu/SPARK-25196.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-10 15:17:46 -07:00
Yuming Wang 470313e660 [SPARK-27118][SQL] Upgrade Hive Metastore Client to the latest versions for Hive 1.0.x/1.1.x
## What changes were proposed in this pull request?

Hive 1.1.1 and Hive 1.0.1 released. We should upgrade Hive Metastore Client version.

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329444&styleName=Text&projectId=12310843
https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329557&styleName=Text&projectId=12310843

## How was this patch tested?

N/A

Closes #24040 from wangyum/SPARK-27118.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:50:10 -08:00
Yuming Wang f732647ae4 [SPARK-27054][BUILD][SQL] Remove the Calcite dependency
## What changes were proposed in this pull request?

Calcite is only used for [runSqlHive](02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L699-L705)) when `hive.cbo.enable=true`([SemanticAnalyzer](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java#L278-L280)).
So we can disable `hive.cbo.enable` and remove Calcite dependency.

## How was this patch tested?

Exist tests

Closes #23970 from wangyum/SPARK-27054.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:34:24 -08:00
Shixiong Zhu 6e1c0827ec
[SPARK-27111][SS] Fix a race that a continuous query may fail with InterruptedException
## What changes were proposed in this pull request?

Before a Kafka consumer gets assigned with partitions, its offset will contain 0 partitions. However, runContinuous will still run and launch a Spark job having 0 partitions. In this case, there is a race that epoch may interrupt the query execution thread after `lastExecution.toRdd`, and either `epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` or the next `runContinuous` will get interrupted unintentionally.

To handle this case, this PR has the following changes:

- Clean up the resources in `queryExecutionThread.runUninterruptibly`. This may increase the waiting time of `stop` but should be minor because the operations here are very fast (just sending an RPC message in the same process and stopping a very simple thread).
- Clear the interrupted status at the end so that it won't impact the `runContinuous` call. We may clear the interrupted status set by `stop`, but it doesn't affect the query termination because `runActivatedStream` will check `state` and exit accordingly.

I also updated the clean up codes to make sure exceptions thrown from `epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` won't stop the clean up.

## How was this patch tested?

Jenkins

Closes #24034 from zsxwing/SPARK-27111.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-03-09 14:26:58 -08:00
CodeGod a29df5fa02 [SPARK-27080][SQL] bug fix: mergeWithMetastoreSchema with uniform lower case comparison
## What changes were proposed in this pull request?
When reading parquet file with merging metastore schema and file schema, we should compare field names using uniform case. In current implementation, lowercase is used but one omission. And this patch fix it.

## How was this patch tested?
Unit test

Closes #24001 from codeborui/mergeSchemaBugFix.

Authored-by: CodeGod <>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-09 21:28:10 +08:00
Kris Mok 57ae251f75 [SPARK-27097] Avoid embedding platform-dependent offsets literally in whole-stage generated code
## What changes were proposed in this pull request?

Spark SQL performs whole-stage code generation to speed up query execution. There are two steps to it:
- Java source code is generated from the physical query plan on the driver. A single version of the source code is generated from a query plan, and sent to all executors.
  - It's compiled to bytecode on the driver to catch compilation errors before sending to executors, but currently only the generated source code gets sent to the executors. The bytecode compilation is for fail-fast only.
- Executors receive the generated source code and compile to bytecode, then the query runs like a hand-written Java program.

In this model, there's an implicit assumption about the driver and executors being run on similar platforms. Some code paths accidentally embedded platform-dependent object layout information into the generated code, such as:
```java
Platform.putLong(buffer, /* offset */ 24, /* value */ 1);
```
This code expects a field to be at offset +24 of the `buffer` object, and sets a value to that field.
But whole-stage code generation generally uses platform-dependent information from the driver. If the object layout is significantly different on the driver and executors, the generated code can be reading/writing to wrong offsets on the executors, causing all kinds of data corruption.

One code pattern that leads to such problem is the use of `Platform.XXX` constants in generated code, e.g. `Platform.BYTE_ARRAY_OFFSET`.

Bad:
```scala
val baseOffset = Platform.BYTE_ARRAY_OFFSET
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will embed the value of `Platform.BYTE_ARRAY_OFFSET` on the driver into the generated code.

Good:
```scala
val baseOffset = "Platform.BYTE_ARRAY_OFFSET"
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will generate the offset symbolically -- `Platform.putLong(buffer, Platform.BYTE_ARRAY_OFFSET, value)`, which will be able to pick up the correct value on the executors.

Caveat: these offset constants are declared as runtime-initialized `static final` in Java, so they're not compile-time constants from the Java language's perspective. It does lead to a slightly increased size of the generated code, but this is necessary for correctness.

NOTE: there can be other patterns that generate platform-dependent code on the driver which is invalid on the executors. e.g. if the endianness is different between the driver and the executors, and if some generated code makes strong assumption about endianness, it would also be problematic.

## How was this patch tested?

Added a new test suite `WholeStageCodegenSparkSubmitSuite`. This test suite needs to set the driver's extraJavaOptions to force the driver and executor use different Java object layouts, so it's run as an actual SparkSubmit job.

Authored-by: Kris Mok <kris.mokdatabricks.com>

Closes #24031 from gatorsmile/cherrypickSPARK-27097.

Lead-authored-by: Kris Mok <kris.mok@databricks.com>
Co-authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-09 01:20:32 +00:00
Sunitha Kambhampati bd2710bd79 [MINOR][SQL] Fix the typo in the spark.sql.extensions conf doc
## What changes were proposed in this pull request?
Fix the  typo (missing the s)  in the class name (SparkSessionExtensions)  in the doc for Spark conf spark.sql.extensions.

## How was this patch tested?
Verified by checking that the configuration doc shows up correctly in spark-shell using the SET -v

Closes #24020 from skambha/fixnametypo.

Authored-by: Sunitha Kambhampati <skambha@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-09 08:51:19 +09:00
SongYadong 14b1312727 [SPARK-27103][SQL][MINOR] List SparkSql reserved keywords in alphabet order
## What changes were proposed in this pull request?

This PR tries to correct spark-sql reserved keywords' position in list if they are not in alphabetical order.
In test suite some repeated words are removed. Also some comments are added for remind.

## How was this patch tested?

Existing unit tests.

Closes #23985 from SongYadong/sql_reserved_alphabet.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-08 10:51:39 -08:00
wangguangxin.cn d3d9c7bb0a [SPARK-27079][MINOR][SQL] Fix typo & Remove useless imports & Add missing override annotation
## What changes were proposed in this pull request?

1. Fix two typos
2. Remove useless imports in `CSVExprUtils.scala`
3. Add missing `override` annotation

## How was this patch tested?

test by existing uts

Closes #24000 from WangGuangxin/SPARK-27079.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-08 12:14:04 -06:00
Ryan Blue 6170e40c15 [SPARK-24252][SQL] Add v2 catalog plugin system
## What changes were proposed in this pull request?

This adds a v2 API for adding new catalog plugins to Spark.

* Catalog implementations extend `CatalogPlugin` and are loaded via reflection, similar to data sources
* `Catalogs` loads and initializes catalogs using configuration from a `SQLConf`
* `CaseInsensitiveStringMap` is used to pass configuration to `CatalogPlugin` via `initialize`

Catalogs are configured by adding config properties starting with `spark.sql.catalog.(name)`. The name property must specify a class that implements `CatalogPlugin`. Other properties under the namespace (`spark.sql.catalog.(name).(prop)`) are passed to the provider during initialization along with the catalog name.

This replaces #21306, which will be implemented in two multiple parts: the catalog plugin system (this commit) and specific catalog APIs, like `TableCatalog`.

## How was this patch tested?

Added test suites for `CaseInsensitiveStringMap` and for catalog loading.

Closes #23915 from rdblue/SPARK-24252-add-v2-catalog-plugins.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 19:31:49 +08:00
Yuming Wang 2036074b99 [SPARK-26004][SQL] InMemoryTable support StartsWith predicate push down
## What changes were proposed in this pull request?

[SPARK-24638](https://issues.apache.org/jira/browse/SPARK-24638) adds support for Parquet file `StartsWith` predicate push down.
`InMemoryTable` can also support this feature.

This is an example to explain how it works, Imagine that the `id` column stored as below:

Partition ID | lowerBound | upperBound
-- | -- | --
p1 | '1' | '9'
p2 | '10' | '19'
p3 | '20' | '29'
p4 | '30' | '39'
p5 | '40' | '49'

A filter ```df.filter($"id".startsWith("2"))``` or ```id like '2%'```
then we substr lowerBound and upperBound:

Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2"))
-- | -- | --
p1 | '1' | '9'
p2 | '1' | '1'
p3 | '2' | '2'
p4 | '3' | '3'
p5 | '4' | '4'

We can see that we only need to read `p1` and `p3`.

## How was this patch tested?

 unit tests and benchmark tests

benchmark test result:
```
================================================================================================
Pushdown benchmark for StringStartsWith
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.12.6
Intel(R) Core(TM) i7-7820HQ CPU  2.90GHz
StringStartsWith filter: (value like '10%'): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InMemoryTable Vectorized                    12068 / 14198          1.3         767.3       1.0X
InMemoryTable Vectorized (Pushdown)           5457 / 8662          2.9         347.0       2.2X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.12.6
Intel(R) Core(TM) i7-7820HQ CPU  2.90GHz
StringStartsWith filter: (value like '1000%'): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InMemoryTable Vectorized                      5246 / 5355          3.0         333.5       1.0X
InMemoryTable Vectorized (Pushdown)           2185 / 2346          7.2         138.9       2.4X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.12.6
Intel(R) Core(TM) i7-7820HQ CPU  2.90GHz
StringStartsWith filter: (value like '786432%'): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InMemoryTable Vectorized                      5112 / 5312          3.1         325.0       1.0X
InMemoryTable Vectorized (Pushdown)           2292 / 2522          6.9         145.7       2.2X
```

Closes #23004 from wangyum/SPARK-26004.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 19:18:32 +08:00
Sean Owen 5ebb4b5723 [SPARK-24783][SQL] spark.sql.shuffle.partitions=0 should throw exception
## What changes were proposed in this pull request?

Throw an exception if spark.sql.shuffle.partitions=0
This takes over https://github.com/apache/spark/pull/23835

## How was this patch tested?

Existing tests.

Closes #24008 from srowen/SPARK-24783.2.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: WindCanDie <491237260@qq.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-08 14:09:53 +09:00
Jungtaek Lim (HeartSaVioR) d8f77e11a4 [SPARK-27001][SQL][FOLLOWUP] Address primitive array type for serializer
## What changes were proposed in this pull request?

This is follow-up PR which addresses review comment in PR for SPARK-27001:
https://github.com/apache/spark/pull/23908#discussion_r261511454

This patch proposes addressing primitive array type for serializer - instead of handling it to generic one, Spark now handles it efficiently as primitive array.

## How was this patch tested?

UT modified to include primitive array.

Closes #24015 from HeartSaVioR/SPARK-27001-FOLLOW-UP-java-primitive-array.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 11:54:04 +08:00
Yuming Wang 43dcb91a4c [SPARK-19678][FOLLOW-UP][SQL] Add behavior change test when table statistics are incorrect
## What changes were proposed in this pull request?

Since Spark 2.2.0 ([SPARK-19678](https://issues.apache.org/jira/browse/SPARK-19678)), the below SQL changed from `broadcast join` to `sort merge join`:
```sql
-- small external table with incorrect statistics
CREATE EXTERNAL TABLE t1(c1 int)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
WITH SERDEPROPERTIES (
  'serialization.format' = '1'
)
STORED AS
  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION 'file:///tmp/t1'
TBLPROPERTIES (
'rawDataSize'='-1', 'numFiles'='0', 'totalSize'='0', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='-1'
);

-- big table
CREATE TABLE t2 (c1 int)
LOCATION 'file:///tmp/t2'
TBLPROPERTIES (
'rawDataSize'='23437737', 'numFiles'='12222', 'totalSize'='333442230', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='443442223'
);

explain SELECT t1.c1 FROM t1 INNER JOIN t2 ON t1.c1 = t2.c1;
```
This pr add a test case for this behavior change.

## How was this patch tested?

unit tests

Closes #24003 from wangyum/SPARK-19678.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 11:47:49 +08:00
Yuming Wang d70b6a39e1 [MINOR][BUILD] Add 2 maven properties(hive.classifier and hive.parquet.group)
## What changes were proposed in this pull request?

This pr adds 2 maven properties to help us upgrade the built-in Hive.

| Property Name | Default | In future |
| ------ | ------ | ------ |
| hive.classifier | (none) | core |
| hive.parquet.group | com.twitter | org.apache.parquet |

## How was this patch tested?

existing tests

Closes #23996 from wangyum/add_2_maven_properties.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-07 16:46:07 -06:00
Shahid 713646ddc2 [SPARK-27075] Remove duplicate execution tag parameters from the url, when accessing the execution table in the SQL page
## What changes were proposed in this pull request?

When we sort any columns in the execution table of the SQL page in the WEBUI, it throws IllegalArgumentException. The root cause is that,  in the url, we are duplicating the execution tag parameters in the 'parameterPath'. Actually we should filter out the executionTag related entries while getting the 'parameterOtherTable'
e9e8bb33ef/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala (L161-L163)
e9e8bb33ef/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala (L241)
e9e8bb33ef/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala (L263-L266)

## How was this patch tested?
Manually tested
Test steps:
Sort any column in the sql page execution table
Before fix:
![screenshot from 2019-03-07 01-38-17](https://user-images.githubusercontent.com/23054875/53913261-f0b69580-4080-11e9-88ea-f238b47a21d5.png)

After fix:
![screenshot from 2019-03-07 02-01-40](https://user-images.githubusercontent.com/23054875/53913285-01670b80-4081-11e9-81b6-78cdbf5a0817.png)

Closes #23994 from shahidki31/SPARK-27075.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-07 12:52:46 -08:00
Takeshi Yamamuro 315c95c399 [SPARK-25863][SPARK-21871][SQL] Check if code size statistics is empty or not in updateAndGetCompilationStats
## What changes were proposed in this pull request?
`CodeGenerator.updateAndGetCompilationStats` throws an unsupported exception for empty code size statistics. This pr added code to check if it is empty or not.

## How was this patch tested?
Pass Jenkins.

Closes #23947 from maropu/SPARK-21871-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-07 17:25:22 +09:00
Dilip Biswal a0e26cffc5 [MINOR][SQL][TEST] Include usage example for generating output for single test in SQLQueryTestSuite
## What changes were proposed in this pull request?
This is a very minor pr to include the usage example to generate output for single test in SQLQueryTestSuite. I tried to deduce it from the existing example and ran into a scenario
where sbt is simply looping to run the same test over and over again. Here is the example
of running a single test.

```
build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql"
```
I tried to generate the output for a single test by prepending `SPARK_GENERATE_GOLDEN_FILES=1` like following
```
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "~sql/test-only *SQLQueryTestSuite -- -z describe.sql"
```
In this case i found that sbt is looping trying to run describe.sql over and over again as we are running the test in on continuous mode (because of `~` prefix ) where it detects a change in
the generated result file which in turn triggers a build and test. I have included an example where
we don't run it in continuous mode when generating the output. Hopefully it saves other developers some time.
## How was this patch tested?
Verified manually in my dev setup.

Closes #23995 from dilipbiswal/dkb_sqlquerytest_usage.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-07 13:06:23 +09:00
Gengliang Wang a543f917e0 [SPARK-27049][SQL] Create util class to support handling partition values in file source V2
## What changes were proposed in this pull request?

While I am migrating other data sources, I find that we should abstract the logic that:
1. converting safe `InternalRow`s into `UnsafeRow`s
2. appending partition values to the end of the result row if existed

This PR proposes to support handling partition values in file source v2 abstraction by adding a util class `PartitionReaderWithPartitionValues`.

## How was this patch tested?

Existing unit tests

Closes #23987 from gengliangwang/SPARK-27049.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-07 11:24:15 +08:00
Yuming Wang 32848eecc5 [SPARK-27078][SQL] Fix NoSuchFieldError when read Hive materialized views
## What changes were proposed in this pull request?

This pr fix `NoSuchFieldError` when reading Hive materialized views from Hive 2.3.4.

How to reproduce:
Hive side:
```sql
CREATE TABLE materialized_view_tbl (key INT);
CREATE MATERIALIZED VIEW view_1 DISABLE REWRITE AS SELECT * FROM materialized_view_tbl;
```
Spark side:
```java
bin/spark-sql --conf spark.sql.hive.metastore.version=2.3.4 --conf spark.sql.hive.metastore.jars=maven

spark-sql> select * from view_1;
19/03/05 19:55:37 ERROR SparkSQLDriver: Failed in [select * from view_1]
java.lang.NoSuchFieldError: INDEX_TABLE
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$3(HiveClientImpl.scala:438)
	at scala.Option.map(Option.scala:163)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$1(HiveClientImpl.scala:370)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:277)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:215)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:214)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:260)
	at org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:368)
```

## How was this patch tested?

unit tests

Closes #23984 from wangyum/SPARK-24360.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-06 16:56:32 -08:00
Maxim Gekk 9513d82edd [SPARK-27057][SQL] Common trait for limit exec operators
## What changes were proposed in this pull request?

I would like to refactor `limit.scala` slightly and introduce common trait `LimitExec` for `CollectLimitExec` and `BaseLimitExec` (`LocalLimitExec` and `GlobalLimitExec`). This will allow to distinguish those operators from others, and to get the `limit` value without casting to concrete class.

## How was this patch tested?

by existing test suites.

Closes #23976 from MaxGekk/limit-exec.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-07 08:47:52 +08:00
Shahid 62fd133f74 [SPARK-27019][SQL][WEBUI] onJobStart happens after onExecutionEnd shouldn't overwrite kvstore
## What changes were proposed in this pull request?
Currently, when the event reordering happens, especially onJobStart event come after onExecutionEnd event, SQL page in the UI displays weirdly.(for eg:test mentioned in JIRA and also this issue randomly occurs when the TPCDS query  fails due to broadcast timeout etc.)

The reason is that, In the SQLAppstatusListener, we remove the liveExecutions entry once the execution ends. So, if a jobStart event come after that, then we create a new liveExecution entry corresponding to the execId. Eventually this will overwrite the kvstore and UI displays confusing entries.

## How was this patch tested?

Added UT, Also manually tested with the eventLog, provided in the jira, of the failed query.

Before fix:
![screenshot from 2019-03-03 03-05-52](https://user-images.githubusercontent.com/23054875/53687929-53e2b800-3d61-11e9-9dca-620fa41e605c.png)

After fix:
![screenshot from 2019-03-03 02-40-18](https://user-images.githubusercontent.com/23054875/53687928-4f1e0400-3d61-11e9-86aa-584646ac68f9.png)

Closes #23939 from shahidki31/SPARK-27019.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-06 14:02:30 -08:00
Udbhav30 9bddf7180e [SPARK-24669][SQL] Invalidate tables in case of DROP DATABASE CASCADE
## What changes were proposed in this pull request?
Before dropping database refresh the tables of that database, so as to refresh all cached entries associated with those tables.
We follow the same when dropping a table.

## How was this patch tested?
UT is added

Closes #23905 from Udbhav30/SPARK-24669.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-06 09:06:10 -08:00
Maxim Gekk 6001258398 [SPARK-27035][SQL] Get more precise current time
## What changes were proposed in this pull request?

In the PR, I propose to replace `System.currentTimeMillis()` by `Instant.now()` in the `CurrentTimestamp` expression. `Instant.now()` uses the best available clock in the system to take current time. See [JDK-8068730](https://bugs.openjdk.java.net/browse/JDK-8068730) for more details. In JDK8, `Instant.now()` provides results with millisecond resolution but starting from JDK9 resolution of results is increased up to microseconds.

## How was this patch tested?

The changes were tested by `DateTimeUtilsSuite` and by `DateFunctionsSuite`.

Closes #23945 from MaxGekk/current-time.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-06 08:32:16 -06:00
Maxim Gekk 9b55722161 [SPARK-27031][SQL] Avoid double formatting in timestampToString
## What changes were proposed in this pull request?

Removed unnecessary conversion of microseconds in `DateTimeUtils.timestampToString` to `java.sql.Timestamp` which aims to output fraction of seconds by casting it to string. This was replaced by special `TimestampFormatter` which appends the fraction formatter to `DateTimeFormatterBuilder`: `appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)`. The former one means trailing zeros in second's fraction should be truncated while formatting.

## How was this patch tested?

By existing test suites like `CastSuite`, `DateTimeUtilsSuite`, `JDBCSuite`, and by new test in `TimestampFormatterSuite`.

Closes #23936 from MaxGekk/timestamp-to-string.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-06 08:26:59 -06:00
Liang-Chi Hsieh 83857496e5 [SPARK-27043][SQL] Add ORC nested schema pruning benchmarks
## What changes were proposed in this pull request?

We have benchmark of nested schema pruning, but only for Parquet. This adds similar benchmark for ORC. This is used with nested schema pruning of ORC.

## How was this patch tested?

Added test.

Closes #23955 from viirya/orc-nested-schema-pruning-benchmark.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-05 11:12:57 -08:00
Takeshi Yamamuro 4490fd0ff0 [SPARK-27001][SQL][FOLLOW-UP] Drop Serializable in WalkedTypePath
## What changes were proposed in this pull request?
This pr tried to drop `Serializable` in `WalkedTypePath`.

## How was this patch tested?
Pass Jenkins.

Closes #23973 from maropu/SPARK-27001-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-05 23:05:50 +08:00
Yuming Wang 940626b724 [SPARK-15095][FOLLOW-UP][SQL] Remove HiveSessionHook related code from ThriftServer
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/12881 removed `HiveSessionHook`. But there are still some code related to `HiveSessionHook`.
This PR removes all `HiveSessionHook` related code.

## How was this patch tested?

manual tests

Closes #23957 from wangyum/SPARK-15095.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-05 07:42:25 -06:00
Anton Okolnychyi 0c23a39384 [SPARK-26205][SQL] Optimize InSet Expression for bytes, shorts, ints, dates
## What changes were proposed in this pull request?

This PR optimizes `InSet` expressions for byte, short, integer, date types. It is a follow-up on PR #21442 from dbtsai.

`In` expressions are compiled into a sequence of if-else statements, which results in O\(n\) time complexity. `InSet` is an optimized version of `In`, which is supposed to improve the performance if all values are literals and the number of elements is big enough. However, `InSet` actually worsens the performance in many cases due to various reasons.

The main idea of this PR is to use Java `switch` statements to significantly improve the performance of `InSet` expressions for bytes, shorts, ints, dates. All `switch` statements are compiled into `tableswitch` and `lookupswitch` bytecode instructions. We will have O\(1\) time complexity if our case values are compact and `tableswitch` can be used. Otherwise, `lookupswitch` will give us O\(log n\).

Locally, I tried Spark `OpenHashSet` and primitive collections from `fastutils` in order to solve the boxing issue in `InSet`. Both options significantly decreased the memory consumption and `fastutils` improved the time compared to `HashSet` from Scala. However, the switch-based approach was still more than two times faster even on 500+ non-compact elements.

I also noticed that applying the switch-based approach on less than 10 elements gives a relatively minor improvement compared to the if-else approach. Therefore, I placed the switch-based logic into `InSet` and added a new config to track when it is applied. Even if we migrate to primitive collections at some point, the switch logic will be still faster unless the number of elements is really big. Another option is to have a separate `InSwitch` expression. However, this would mean we need to modify other places (e.g., `DataSourceStrategy`).

See [here](https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-3.html#jvms-3.10) and [here](https://stackoverflow.com/questions/10287700/difference-between-jvms-lookupswitch-and-tableswitch) for more information.

This PR does not cover long values as Java `switch` statements cannot be used on them. However, we can have a follow-up PR with an approach similar to binary search.

## How was this patch tested?

There are new tests that verify the logic of the proposed optimization.

The performance was evaluated using existing benchmarks. This PR was also tested on an EC2 instance (OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 4.14.77-70.59.amzn1.x86_64, Intel(R) Xeon(R) CPU E5-2686 v4  2.30GHz).

## Notes

- [This link](http://hg.openjdk.java.net/jdk8/jdk8/langtools/file/30db5e0aaf83/src/share/classes/com/sun/tools/javac/jvm/Gen.java#l1153) contains source code that decides between `tableswitch` and `lookupswitch`. The logic was re-used in the benchmarks. See the `isLookupSwitch` method.

Closes #23171 from aokolnychyi/spark-26205.

Lead-authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-04 15:40:04 -08:00