Commit graph

3928 commits

Author SHA1 Message Date
Gengliang Wang c88df2ccf6 [SPARK-28331][SQL] Catalogs.load() should be able to load built-in catalogs
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

Closes #25094 from gengliangwang/fixCatalogLoad.

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

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

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

## How was this patch tested?

Added UT

Closes #25253 from mgaido91/SPARK-28470.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-08 08:10:21 +09:00
Wenchen Fan 469423f338 [SPARK-28595][SQL] explain should not trigger partition listing
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

existing tests

Closes #25328 from cloud-fan/ui.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-07 19:14:25 +08:00
mcheah 44e607e921 [SPARK-28238][SQL] Implement DESCRIBE TABLE for Data Source V2 Tables
## What changes were proposed in this pull request?

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

## How was this patch tested?

Added unit tests to `DataSourceV2SQLSuite`.

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

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-07 14:26:45 +08:00
Nik Vanderhoof 9e931e787d [SPARK-27905][SQL] Add higher order function 'forall'
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

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

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

Closes #24761 from nvander1/feature/for_all.

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

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

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

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

## How was this patch tested?

Updated `extract.sql`.

Closes #25367 from MaxGekk/extract-ext.

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

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

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

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

## How was this patch tested?

Existing tests should cover.

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

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-05 23:36:35 -07:00
Jungtaek Lim (HeartSaVioR) 128ea37bda [SPARK-28601][CORE][SQL] Use StandardCharsets.UTF_8 instead of "UTF-8" string representation, and get rid of UnsupportedEncodingException
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Existing unit tests.

Closes #25335 from HeartSaVioR/SPARK-28601.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-05 20:45:54 -07:00
Wenchen Fan 6fb79af48c [SPARK-28344][SQL] detect ambiguous self-join and fail the query
## What changes were proposed in this pull request?

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

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

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

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

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

## How was this patch tested?

existing tests and new test cases

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

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-06 10:06:36 +08:00
Ryan Blue 0345f1174d [SPARK-27661][SQL] Add SupportsNamespaces API
## What changes were proposed in this pull request?

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

## How was this patch tested?

API only. Existing tests for regressions.

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

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-04 21:29:40 -07:00
Xiao Li 10d4ffd577 [SPARK-28532][SPARK-28530][SQL][FOLLOWUP] Inline doc for FixedPoint(1) batches "Subquery" and "Join Reorder"
## What changes were proposed in this pull request?
Explained why "Subquery" and "Join Reorder" optimization batches should be `FixedPoint(1)`, which was introduced in SPARK-28532 and SPARK-28530.

## How was this patch tested?

Existing UTs.

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

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

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

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

## How was this patch tested?

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

Closes #25279 from srowen/SPARK-28519.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-02 10:55:44 -05:00
Liang-Chi Hsieh 77c7e91e02 [SPARK-28445][SQL][PYTHON] Fix error when PythonUDF is used in both group by and aggregate expression
## What changes were proposed in this pull request?

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

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

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

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

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

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

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

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

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

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

## How was this patch tested?

Added tests.

Closes #25215 from viirya/SPARK-28445.

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

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

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

## How was this patch tested?

unit tests

Closes #25261 from wangyum/SPARK-28521.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-01 18:02:50 -05:00
Marco Gaido ee41001949 [SPARK-26218][SQL] Overflow on arithmetic operations returns incorrect result
## What changes were proposed in this pull request?

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

## How was this patch tested?

added UT + existing UTs (improved)

Closes #21599 from mgaido91/SPARK-24598.

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

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

## How was this patch tested?

unit tests

Closes #25314 from wangyum/SPARK-28581.

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

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

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

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

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

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

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

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

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

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

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

spark-sql> select true is unknown, true is not unknown;
false	true
```
**Note**: A null input is treated as the logical value "unknown".

## How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #25074 from beliefer/ansi-sql-boolean-test.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-30 23:59:50 -07:00
Dilip Biswal ee3c1c777d [SPARK-28375][SQL] Make pullupCorrelatedPredicate idempotent
## What changes were proposed in this pull request?

This PR makes the optimizer rule PullupCorrelatedPredicates idempotent.
## How was this patch tested?

A new test PullupCorrelatedPredicatesSuite

Closes #25268 from dilipbiswal/pr-25164.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-30 16:29:24 -07:00
Maxim Gekk caa23e3efd [SPARK-28459][SQL] Add make_timestamp function
## What changes were proposed in this pull request?

New function `make_timestamp()` takes 6 columns `year`, `month`, `day`, `hour`, `min`, `sec` + optionally `timezone`, and makes new column of the `TIMESTAMP` type. If values in the input columns are `null` or out of valid ranges, the function returns `null`. Valid ranges are:
- `year` - `[1, 9999]`
- `month` - `[1, 12]`
- `day` - `[1, 31]`
- `hour` - `[0, 23]`
- `min` - `[0, 59]`
- `sec` - `[0, 60]`. If the `sec` argument equals to 60, the seconds field is set to 0 and 1 minute is added to the final timestamp.
- `timezone` - an identifier of timezone. Actual database of timezones can be found there: https://www.iana.org/time-zones.

Also constructed timestamp must be valid otherwise `make_timestamp` returns `null`.

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

Here is an example:
```sql
select make_timestamp(2014, 12, 28, 6, 30, 45.887);
  2014-12-28 06:30:45.887
select make_timestamp(2014, 12, 28, 6, 30, 45.887, 'CET');
  2014-12-28 10:30:45.887
select make_timestamp(2019, 6, 30, 23, 59, 60)
  2019-07-01 00:00:00
```

Returned value has Spark Catalyst type `TIMESTAMP` which is similar to Oracle's `TIMESTAMP WITH LOCAL TIME ZONE` (see https://docs.oracle.com/cd/B28359_01/server.111/b28298/ch4datetime.htm#i1006169) where data is stored in the session time zone, and the time zone offset is not stored as part of the column data. When users retrieve the data, Spark returns it in the session time zone specified by the SQL config `spark.sql.session.timeZone`.

## How was this patch tested?

Added new tests to `DateExpressionsSuite`, and uncommented a test for `make_timestamp` in `pgSQL/timestamp.sql`.

Closes #25220 from MaxGekk/make_timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-29 11:00:08 -07:00
Lee Dongjin d98aa2a184 [MINOR] Trivial cleanups
These are what I found during working on #22282.

- Remove unused value: `UnsafeArraySuite#defaultTz`
- Remove redundant new modifier to the case class, `KafkaSourceRDDPartition`
- Remove unused variables from `RDD.scala`
- Remove trailing space from `structured-streaming-kafka-integration.md`
- Remove redundant parameter from `ArrowConvertersSuite`: `nullable` is `true` by default.
- Remove leading empty line: `UnsafeRow`
- Remove trailing empty line: `KafkaTestUtils`
- Remove unthrown exception type: `UnsafeMapData`
- Replace unused declarations: `expressions`
- Remove duplicated default parameter: `AnalysisErrorSuite`
- `ObjectExpressionsSuite`: remove duplicated parameters, conversions and unused variable

Closes #25251 from dongjinleekr/cleanup/201907.

Authored-by: Lee Dongjin <dongjin@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 23:38:02 +09:00
Dongjoon Hyun 18156d5503 [SPARK-28086][SQL] Add a function alias random for Rand
## What changes were proposed in this pull request?

This PR aims to add a SQL function alias `random` to the existing `rand` function.
Please note that this adds the alias to SQL layer only because this is for PostgreSQL feature parity.

- [PostgreSQL Random function](https://www.postgresql.org/docs/11/functions-math.html)
- [SPARK-23160 Port window.sql](https://github.com/apache/spark/pull/24881/files#diff-14489bae6b27814d4cde0456a7ae75c8R702)
- [SPARK-28406 Port union.sql](https://github.com/apache/spark/pull/25163/files#diff-23a3430e0e1ff88830cbb43701da1f2cR402)

## How was this patch tested?

Manual.
```sql
spark-sql> DESCRIBE FUNCTION random;
Function: random
Class: org.apache.spark.sql.catalyst.expressions.Rand
Usage: random([seed]) - Returns a random value with independent and identically distributed (i.i.d.) uniformly distributed values in [0, 1).
```

Closes #25282 from dongjoon-hyun/SPARK-28086.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 20:17:30 +09:00
Maxim Gekk a5a5da78cf [SPARK-28471][SQL] Replace yyyy by uuuu in date-timestamp patterns without era
## What changes were proposed in this pull request?

In the PR, I propose to use `uuuu` for years instead of `yyyy` in date/timestamp patterns without the era pattern `G` (https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html). **Parsing/formatting of positive years (current era) will be the same.** The difference is in formatting negative years belong to previous era - BC (Before Christ).

I replaced the `yyyy` pattern by `uuuu` everywhere except:
1. Test, Suite & Benchmark. Existing tests must work as is.
2. `SimpleDateFormat` because it doesn't support the `uuuu` pattern.
3. Comments and examples (except comments related to already replaced patterns).

Before the changes, the year of common era `100` and the year of BC era `-99`, showed similarly as `100`.  After the changes negative years will be formatted with the `-` sign.

Before:
```Scala
scala> Seq(java.time.LocalDate.of(-99, 1, 1)).toDF().show
+----------+
|     value|
+----------+
|0100-01-01|
+----------+
```

After:
```Scala
scala> Seq(java.time.LocalDate.of(-99, 1, 1)).toDF().show
+-----------+
|      value|
+-----------+
|-0099-01-01|
+-----------+
```

## How was this patch tested?

By existing test suites, and added tests for negative years to `DateFormatterSuite` and `TimestampFormatterSuite`.

Closes #25230 from MaxGekk/year-pattern-uuuu.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-28 20:36:36 -07:00
Dongjoon Hyun a428f40669 [SPARK-28549][BUILD][CORE][SQL] Use text.StringEscapeUtils instead lang3.StringEscapeUtils
## What changes were proposed in this pull request?

`org.apache.commons.lang3.StringEscapeUtils` was deprecated over two years ago at [LANG-1316](https://issues.apache.org/jira/browse/LANG-1316). There is no bug fixes after that.
```java
/**
 * <p>Escapes and unescapes {code String}s for
 * Java, Java Script, HTML and XML.</p>
 *
 * <p>#ThreadSafe#</p>
 * since 2.0
 * deprecated as of 3.6, use commons-text
 * <a href="https://commons.apache.org/proper/commons-text/javadocs/api-release/org/apache/commons/text/StringEscapeUtils.html">
 * StringEscapeUtils</a> instead
 */
Deprecated
public class StringEscapeUtils {
```

This PR aims to use the latest one from `commons-text` module which has more bug fixes like
[TEXT-100](https://issues.apache.org/jira/browse/TEXT-100), [TEXT-118](https://issues.apache.org/jira/browse/TEXT-118) and [TEXT-120](https://issues.apache.org/jira/browse/TEXT-120) by the following replacement.
```scala
-import org.apache.commons.lang3.StringEscapeUtils
+import org.apache.commons.text.StringEscapeUtils
```

This will add a new dependency to `hadoop-2.7` profile distribution. In `hadoop-3.2` profile, we already have it.
```
+commons-text-1.6.jar
```

## How was this patch tested?

Pass the Jenkins with the existing tests.
- [Hadoop 2.7](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108281)
- [Hadoop 3.2](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108282)

Closes #25281 from dongjoon-hyun/SPARK-28549.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 11:45:29 +09:00
Yuming Wang 9eb541be22 [SPARK-28424][SQL] Support typed interval expression
## What changes were proposed in this pull request?

This PR add support typed `interval` expression:
```sql
spark-sql> select interval 'interval 3 year 1 hour';
interval 3 years 1 hours
spark-sql>
```

Please note that this pr did not add a cast alias for `interval` type like [other types](2d74f14d74/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L529-L541)) because neither PostgreSQL nor Hive supports this syntax.

## How was this patch tested?

unit tests

Closes #25241 from wangyum/SPARK-28424.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-27 14:25:35 -07:00
Yesheng Ma d4e246658a [SPARK-28530][SQL] Cost-based join reorder optimizer batch should be FixedPoint(1)
## What changes were proposed in this pull request?
Since for AQP the cost for joins can change between multiple runs, there is no reason that we have an idempotence enforcement on this optimizer batch. We thus make it `FixedPoint(1)` instead of `Once`.

## How was this patch tested?
Existing UTs.

Closes #25266 from yeshengm/SPARK-28530.

Lead-authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 22:57:39 -07:00
Yesheng Ma e037a11494 [SPARK-28532][SQL] Make optimizer batch "subquery" FixedPoint(1)
## What changes were proposed in this pull request?
In the Catalyst optimizer, the batch subquery actually calls the optimizer recursively. Therefore it makes no sense to enforce idempotence on it and we change this batch to `FixedPoint(1)`.

## How was this patch tested?
Existing UTs.

Closes #25267 from yeshengm/SPARK-28532.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 22:48:42 -07:00
Liang-Chi Hsieh 558dd23601 [SPARK-28441][SQL][PYTHON] Fix error when non-foldable expression is used in correlated scalar subquery
## What changes were proposed in this pull request?

In SPARK-15370, We checked the expression at the root of the correlated subquery, in order to fix count bug. If a `PythonUDF` in in the checking path, evaluating it causes the failure as we can't statically evaluate `PythonUDF`. The Python UDF test added at SPARK-28277 shows this issue.

If we can statically evaluate the expression, we intercept NULL values coming from the outer join and replace them with the value that the subquery's expression like before, if it is not, we replace them with the `PythonUDF` expression, with statically evaluated parameters.

After this, the last query in `udf-except.sql` which throws `java.lang.UnsupportedOperationException` can be run:

```
SELECT t1.k
FROM   t1
WHERE  t1.v <= (SELECT   udf(max(udf(t2.v)))
                FROM     t2
                WHERE    udf(t2.k) = udf(t1.k))
MINUS
SELECT t1.k
FROM   t1
WHERE  udf(t1.v) >= (SELECT   min(udf(t2.v))
                FROM     t2
                WHERE    t2.k = t1.k)
-- !query 2 schema
struct<k:string>
-- !query 2 output
two
```

Note that this issue is also for other non-foldable expressions, like rand. As like PythonUDF, we can't call `eval` on this kind of expressions in optimization. The evaluation needs to defer to query runtime.

## How was this patch tested?

Added tests.

Closes #25204 from viirya/SPARK-28441.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-27 10:38:34 +08:00
Yesheng Ma c93d2dd183 [SPARK-28237][SQL] Enforce Idempotence for Once batches in RuleExecutor
## What changes were proposed in this pull request?
In adaptive query processing (AQE), query plans are optimized on the fly during execution. However, a few `Once` rules can be problematic for such optimization since they can either generate wrong plan/unnecessary intermediate plan nodes.

This PR enforces idempotence for "Once" batches that are supposed to run once. This is a key enabler for AQE re-optimization and can improve robustness for existing optimizer rules.

Once batches that are currently not idempotent are marked in a blacklist. We will submit followup PRs to fix idempotence of these rules.

## How was this patch tested?
Existing UTs. Failing Once rules are temporarily blacklisted.

Closes #25249 from yeshengm/idempotence-checker.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-25 23:44:56 -07:00
Ryan Blue 443904a140 [SPARK-27845][SQL] DataSourceV2: InsertTable
## What changes were proposed in this pull request?

Support multiple catalogs in the following InsertTable use cases:

- INSERT INTO [TABLE] catalog.db.tbl
- INSERT OVERWRITE TABLE catalog.db.tbl

Support matrix:

Overwrite|Partitioned Table|Partition Clause |Partition Overwrite Mode|Action
---------|-----------------|-----------------|------------------------|-----
false|*|*|*|AppendData
true|no|(empty)|*|OverwriteByExpression(true)
true|yes|p1,p2 or p1 or p2 or (empty)|STATIC|OverwriteByExpression(true)
true|yes|p2,p2 or p1 or p2 or (empty)|DYNAMIC|OverwritePartitionsDynamic
true|yes|p1=23,p2=3|*|OverwriteByExpression(p1=23 and p2=3)
true|yes|p1=23,p2 or p1=23|STATIC|OverwriteByExpression(p1=23)
true|yes|p1=23,p2 or p1=23|DYNAMIC|OverwritePartitionsDynamic

Notes:
- Assume the partitioned table has 2 partitions: p1 and p2.
- `STATIC` is the default Partition Overwrite Mode for data source tables.
- DSv2 tables currently do not support `IfPartitionNotExists`.

## How was this patch tested?

New tests.
All existing catalyst and sql/core tests.

Closes #24832 from jzhuge/SPARK-27845-pr.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-07-25 15:05:51 -07:00
Gengliang Wang b367b323d2 [SPARK-28497][SQL] Disallow upcasting complex data types to string type
## What changes were proposed in this pull request?

In the current implementation. complex types like Array/Map/StructType are allowed to upcast as StringType.
This is not safe casting. We should disallow it.

## How was this patch tested?

Update the existing test case

Closes #25242 from gengliangwang/fixUpCastStringType.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-25 20:55:01 +09:00
Yuming Wang 045191e610 [SPARK-28293][SQL] Implement Spark's own GetTableTypesOperation
## What changes were proposed in this pull request?

The table type is from Hive now. This will have some issues. For example, we don't support `index_table`, different Hive supports different table types:
Build with Hive 1.2.1:
![image](https://user-images.githubusercontent.com/5399861/60792689-be38b880-a198-11e9-82b8-868992a505e3.png)
Build with Hive 2.3.5:
![image](https://user-images.githubusercontent.com/5399861/60792727-d4467900-a198-11e9-952c-210bb7bb3bed.png)

This pr implement Spark's own `GetTableTypesOperation`.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/60793368-2a67ec00-a19a-11e9-9511-c67483dcc370.png)

Closes #25073 from wangyum/SPARK-28293.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-24 11:27:30 -07:00
Yuming Wang d67b98ea01 [SPARK-28435][SQL] Support accepting the interval keyword in the schema string
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/7355 add support casting between IntervalType and StringType for scala interface:
```scala
import org.apache.spark.sql.types._
import org.apache.spark.sql.catalyst.expressions._

Cast(Literal("interval 3 month 1 hours"), CalendarIntervalType).eval()
res0: Any = interval 3 months 1 hours
```
But SQL interface does not support it:
```sql
scala> spark.sql("SELECT CAST('interval 3 month 1 hour' AS interval)").show
org.apache.spark.sql.catalyst.parser.ParseException:
DataType interval is not supported.(line 1, pos 41)

== SQL ==
SELECT CAST('interval 3 month 1 hour' AS interval)
-----------------------------------------^^^

  at org.apache.spark.sql.catalyst.parser.AstBuilder.$anonfun$visitPrimitiveDataType$1(AstBuilder.scala:1931)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:108)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitPrimitiveDataType(AstBuilder.scala:1909)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitPrimitiveDataType(AstBuilder.scala:52)
...
```

This PR add supports accepting the `interval` keyword in the schema string. So that SQL interface can support this feature.

## How was this patch tested?

unit tests

Closes #25189 from wangyum/SPARK-28435.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-23 19:40:57 -07:00
Wenchen Fan e04f696f7f [SPARK-28346][SQL] clone the query plan between analyzer, optimizer and planner
## What changes were proposed in this pull request?

query plan was designed to be immutable, but sometimes we do allow it to carry mutable states, because of the complexity of the SQL system. One example is `TreeNodeTag`. It's a state of `TreeNode` and can be carried over during copy and transform. The adaptive execution framework relies on it to link the logical and physical plans.

This leads to a problem: when we get `QueryExecution#analyzed`, the plan can be changed unexpectedly because it's mutable. I hit a real issue in https://github.com/apache/spark/pull/25107 : I use `TreeNodeTag` to carry dataset id in logical plans. However, the analyzed plan ends up with many duplicated dataset id tags in different nodes. It turns out that, the optimizer transforms the logical plan and add the tag to more nodes.

For example, the logical plan is `SubqueryAlias(Filter(...))`, and I expect only the `SubqueryAlais` has the dataset id tag. However, the optimizer removes `SubqueryAlias` and carries over the dataset id tag to `Filter`. When I go back to the analyzed plan, both `SubqueryAlias` and `Filter` has the dataset id tag, which breaks my assumption.

Since now query plan is mutable, I think it's better to limit the life cycle of a query plan instance. We can clone the query plan between analyzer, optimizer and planner, so that the life cycle is limited in one stage.

## How was this patch tested?

new test

Closes #25111 from cloud-fan/clone.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-23 09:00:39 -07:00
Yuming Wang 022667cea6 [SPARK-28469][SQL] Change CalendarIntervalType's readable string representation from calendarinterval to interval
## What changes were proposed in this pull request?

This PR change `CalendarIntervalType`'s readable string representation from `calendarinterval` to `interval`.

## How was this patch tested?

Existing UT

Closes #25225 from wangyum/SPARK-28469.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-22 20:53:59 -07:00
WeichenXu 185c93e701 [SPARK-28431][SQL] Set maximum error message length in CSV datasource's parsing and writing
## What changes were proposed in this pull request?

Fix CSV datasource to throw `com.univocity.parsers.common.TextParsingException` with large size message, which will make log output consume large disk space.
This issue is troublesome when sometimes we need parse CSV with large size column.

This PR proposes to set CSV parser/writer settings by `setErrorContentLength(1000)` to limit the error message length.

## How was this patch tested?

Manually.

```
val s = "a" * 40 * 1000000
Seq(s).toDF.write.mode("overwrite").csv("/tmp/bogdan/es4196.csv")

spark.read .option("maxCharsPerColumn", 30000000) .csv("/tmp/bogdan/es4196.csv").count
```

**Before:**
The thrown message will include error content of about 30MB size (The column size exceed the max value 30MB, so the error content include the whole parsed content, so it is 30MB).

**After:**
The thrown message will include error content like "...aaa...aa" (the number of 'a' is 1024), i.e. limit the content size to be 1024.

Closes #25184 from WeichenXu123/limit_csv_exception_size.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-23 10:44:59 +09:00
Maxim Gekk 2d74f14d74 [SPARK-28432][SQL] Add make_date function
## What changes were proposed in this pull request?

New function `make_date()` takes 3 columns `year`, `month` and `day`, and makes new column of the `DATE` type. If values in the input columns are `null` or out of valid ranges, the function returns `null`. Valid ranges are:
- `year` - `[1, 9999]`
- `month` - `[1, 12]`
- `day` - `[1, 31]`

Also constructed date must be valid otherwise `make_date` returns `null`.

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

Here is an example:
```sql
select make_date(2013, 7, 15);
2013-07-15
```

## How was this patch tested?

Added new tests to `DateExpressionsSuite`.

Closes #25210 from MaxGekk/make_date-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-22 15:17:06 -07:00
Shixiong Zhu 62e28248f1 [SPARK-28456][SQL] Add a public API Encoder.makeCopy to allow creating Encoder without touching Scala Reflection
## What changes were proposed in this pull request?

Because `Encoder` is not thread safe, the user cannot reuse an `Encoder` in multiple `Dataset`s. However, creating an `Encoder` for a complicated class is slow due to Scala Reflection. To eliminate the cost of Scala Reflection, right now I usually use the private API `ExpressionEncoder.copy` as follows:

```scala
object FooEncoder {
  private lazy val _encoder: ExpressionEncoder[Foo] = ExpressionEncoder[Foo]()
  implicit def encoder: ExpressionEncoder[Foo] = _encoder.copy()
}
```

This PR proposes a new method `makeCopy` in `Encoder` so that the above codes can be rewritten using public APIs.

```scala
object FooEncoder {
  private lazy val _encoder: Encoder[Foo] = Encoders.product[Foo]()
  implicit def encoder: Encoder[Foo] = _encoder.makeCopy
}
```

The method name is consistent with `TreeNode.makeCopy`.

## How was this patch tested?

Jenkins

Closes #25209 from zsxwing/encoder-copy.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-22 12:31:51 +08:00
mcheah 7ed0088539 [SPARK-27724][SQL] Implement REPLACE TABLE and REPLACE TABLE AS SELECT with V2
## What changes were proposed in this pull request?

Implements the `REPLACE TABLE` and `REPLACE TABLE AS SELECT` logical plans. `REPLACE TABLE` is now a valid operation in spark-sql provided that the tables being modified are managed by V2 catalogs.

This also introduces an atomic mix-in that table catalogs can choose to implement. Table catalogs can now implement `TransactionalTableCatalog`. The semantics of this API are that table creation and replacement can be "staged" and then "committed".

On the execution of `REPLACE TABLE AS SELECT`, `REPLACE TABLE`, and `CREATE TABLE AS SELECT`, if the catalog implements transactional operations, the physical plan will use said functionality. Otherwise, these operations fall back on non-atomic variants. For `REPLACE TABLE` in particular, the usage of non-atomic operations can unfortunately lead to inconsistent state.

## How was this patch tested?

Unit tests - multiple additions to `DataSourceV2SQLSuite`.

Closes #24798 from mccheah/spark-27724.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-22 12:08:46 +08:00
Marco Gaido a783690d8a [SPARK-28369][SQL] Honor spark.sql.decimalOperations.nullOnOverflow in ScalaUDF result
## What changes were proposed in this pull request?

When a `ScalaUDF` returns a value which overflows, currently it returns null regardless of the value of the config `spark.sql.decimalOperations.nullOnOverflow`.

The PR makes it respect the above-mentioned config and behave accordingly.

## How was this patch tested?

added UT

Closes #25144 from mgaido91/SPARK-28369.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-07-22 10:39:40 +08:00
Takeshi Yamamuro fced6696a7 [SPARK-28462][SQL][TEST] Add a prefix '*' to non-nullable attribute names in PlanTestBase.comparePlans failures
## What changes were proposed in this pull request?
This pr proposes to add a prefix '*' to non-nullable attribute names in PlanTestBase.comparePlans failures. In the current master, nullability mismatches might generate the same error message for left/right logical plans like this;
```
// This failure message was extracted from #24765
- constraints should be inferred from aliased literals *** FAILED ***
  == FAIL: Plans do not match ===
  !'Join Inner, (two#0 = a#0)                    'Join Inner, (two#0 = a#0)
   :- Filter (isnotnull(a#0) AND (2 <=> a#0))     :- Filter (isnotnull(a#0) AND (2 <=> a#0))
   :  +- LocalRelation <empty>, [a#0, b#0, c#0]   :  +- LocalRelation <empty>, [a#0, b#0, c#0]
   +- Project [2 AS two#0]                        +- Project [2 AS two#0]
      +- LocalRelation <empty>, [a#0, b#0, c#0]      +- LocalRelation <empty>, [a#0, b#0, c#0] (PlanTest.scala:145)
```
With this pr, this error message is changed to one below;
```
- constraints should be inferred from aliased literals *** FAILED ***
  == FAIL: Plans do not match ===
  !'Join Inner, (*two#0 = a#0)                    'Join Inner, (*two#0 = *a#0)
   :- Filter (isnotnull(a#0) AND (2 <=> a#0))     :- Filter (isnotnull(a#0) AND (2 <=> a#0))
   :  +- LocalRelation <empty>, [a#0, b#0, c#0]   :  +- LocalRelation <empty>, [a#0, b#0, c#0]
   +- Project [2 AS two#0]                        +- Project [2 AS two#0]
      +- LocalRelation <empty>, [a#0, b#0, c#0]      +- LocalRelation <empty>, [a#0, b#0, c#0] (PlanTest.scala:145)
```

## How was this patch tested?
N/A

Closes #25213 from maropu/MarkForNullability.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-21 13:34:35 -07:00
Xingbo Jiang 36d7d81d23 [SPARK-27815][SQL][FOLLOWUP][DOC] Update comment that references PushDownPredicate
## What changes were proposed in this pull request?

The optimize rule `PushDownPredicate` has been combined into `PushDownPredicates`, update the comment that references the old rule.

## How was this patch tested?

N/A

Closes #25207 from jiangxb1987/comment.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-20 16:44:28 +09:00
Liang-Chi Hsieh 127bc899ae [SPARK-27707][SQL] Prune unnecessary nested fields from Generate
## What changes were proposed in this pull request?

Performance issue using explode was found when a complex field contains huge array is to get duplicated as the number of exploded array elements. Given example:

```scala
val df = spark.sparkContext.parallelize(Seq(("1",
  Array.fill(M)({
    val i = math.random
    (i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString)
  })))).toDF("col", "arr")
  .selectExpr("col", "struct(col, arr) as st")
  .selectExpr("col", "st.col as col1", "explode(st.arr) as arr_col")
```

The explode causes `st` to be duplicated as many as the exploded elements.

Benchmarks it:

```
[info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4
[info] Intel(R) Core(TM) i7-8750H CPU  2.20GHz
[info] generate big nested struct array:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] generate big nested struct array wholestage off          52668          53162         699          0.0      877803.4       1.0X
[info] generate big nested struct array wholestage on          47261          49093        1125          0.0      787690.2       1.1X
[info]
```

The query plan:
```
== Physical Plan ==
 Project [col#508, st#512.col AS col1#515, arr_col#519]
 +- Generate explode(st#512.arr), [col#508, st#512], false, [arr_col#519]
    +- Project [_1#503 AS col#508, named_struct(col, _1#503, arr, _2#504) AS st#512]
       +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#503, mapobjects(MapObjects_loopValue84, MapObjects_loopIsNull84,      ObjectType(class scala.Tuple4), if (isnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true)))     null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._2, true, false), _3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String,     StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84, MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._3, true,  false), _4, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue84,   MapObjects_loopIsNull84, ObjectType(class scala.Tuple4), true))._4, true, false)), knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, None) AS _2#504]
          +- Scan[obj#534]
```

This patch takes nested column pruning approach to prune unnecessary nested fields. It adds a projection of the needed nested fields as aliases on the child of `Generate`, and substitutes them by alias attributes on the projection on top of `Generate`.

Benchmarks it after the change:
```
 [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4
 [info] Intel(R) Core(TM) i7-8750H CPU  2.20GHz
 [info] generate big nested struct array:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 [info] ------------------------------------------------------------------------------------------------------------------------
 [info] generate big nested struct array wholestage off            311            331          28          0.2        5188.6       1.0X
 [info] generate big nested struct array wholestage on            297            312          15          0.2        4947.3       1.0X
 [info]
```

The query plan:
```
== Physical Plan ==
 Project [col#592, _gen_alias_608#608 AS col1#599, arr_col#603]
 +- Generate explode(st#596.arr), [col#592, _gen_alias_608#608], false, [arr_col#603]
    +- Project [_1#587 AS col#592, named_struct(col, _1#587, arr, _2#588) AS st#596, _1#587 AS _gen_alias_608#608]
       +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(in
 put[0, scala.Tuple2, true]))._1, true, false) AS _1#587, mapobjects(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4),
 if (isnull(lambdavariable(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))) null else named_struct(_1,        staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue102,              MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String,    StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._2,      true, false), _3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString,                                                 knownnotnull(lambdavariable(MapObjects_loopValue102, MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._3, true, false), _4,            staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(lambdavariable(MapObjects_loopValue102,              MapObjects_loopIsNull102, ObjectType(class scala.Tuple4), true))._4, true, false)), knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2,      None) AS _2#588]
          +- Scan[obj#586]
```

This behavior is controlled by a SQL config `spark.sql.optimizer.expression.nestedPruning.enabled`.

## How was this patch tested?

Added benchmark.

Closes #24637 from viirya/SPARK-27707.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-18 23:32:07 -07:00
Maxim Gekk 54e058dff2 [SPARK-28416][SQL] Use java.time API in timestampAddInterval
## What changes were proposed in this pull request?

The `DateTimeUtils.timestampAddInterval` method was rewritten by using Java 8 time API. To add months and microseconds, I used the `plusMonths()` and `plus()` methods of `ZonedDateTime`. Also the signature of `timestampAddInterval()` was changed to accept an `ZoneId` instance instead of `TimeZone`. Using `ZoneId` allows to avoid the conversion `TimeZone` -> `ZoneId` on every invoke of `timestampAddInterval()`.

## How was this patch tested?

By existing test suites `DateExpressionsSuite`, `TypeCoercionSuite` and `CollectionExpressionsSuite`.

Closes #25173 from MaxGekk/timestamp-add-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-18 19:17:23 -04:00
Yuming Wang 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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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 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
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 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
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
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
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
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
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
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
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
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
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
韩田田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
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
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
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
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
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 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
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 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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
Takeshi Yamamuro 68fbbbea4e [SPARK-26965][SQL] Makes ElementAt nullability more precise for array cases
## What changes were proposed in this pull request?
In master, `ElementAt` nullable is always true;
be1cadf16d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala (L1977)

But, If input is an array and foldable, we could make its nullability more precise.
This fix is based on  SPARK-26637(#23566).

## How was this patch tested?
Added tests in `CollectionExpressionsSuite`.

Closes #23867 from maropu/SPARK-26965.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-04 21:27:18 +08:00
Dilip Biswal ad4823c99d [SPARK-19712][SQL] Pushing Left Semi and Left Anti joins through Project, Aggregate, Window, Union etc.
## What changes were proposed in this pull request?
This PR adds support for pushing down LeftSemi and LeftAnti joins below operators such as Project, Aggregate, Window, Union etc.  This is the initial piece of work that will be needed for
the subsequent work 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 a new test suite LeftSemiAntiJoinPushDownSuite.

Closes #23750 from dilipbiswal/SPARK-19712-pushleftsemi.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-04 19:09:24 +08:00
Jungtaek Lim (HeartSaVioR) 34f606678a [SPARK-27001][SQL] Refactor "serializerFor" method between ScalaReflection and JavaTypeInference
## What changes were proposed in this pull request?

This patch proposes refactoring `serializerFor` method between `ScalaReflection` and `JavaTypeInference`, being consistent with what we refactored for `deserializerFor` in #23854.

This patch also extracts the logic on recording walk type path since the logic is duplicated across `serializerFor` and `deserializerFor` with `ScalaReflection` and `JavaTypeInference`.

## How was this patch tested?

Existing tests.

Closes #23908 from HeartSaVioR/SPARK-27001.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-04 10:45:48 +08:00
Dilip Biswal 4a486d6716 [SPARK-26982][SQL] Enhance describe framework to describe the output of a query.
## What changes were proposed in this pull request?
Currently we can use `df.printSchema` to discover the schema information for a query. We should have a way to describe the output schema of a query using SQL interface.

Example:

DESCRIBE SELECT * FROM desc_table
DESCRIBE QUERY SELECT * FROM desc_table
```SQL

spark-sql> create table desc_table (c1 int comment 'c1-comment', c2 decimal comment 'c2-comment', c3 string);

spark-sql> desc select * from desc_table;
c1	int	        c1-comment
c2	decimal(10,0)	c2-comment
c3	string	        NULL

```
## How was this patch tested?
Added a new test under SQLQueryTestSuite and SparkSqlParserSuite

Closes #23883 from dilipbiswal/dkb_describe_query.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-02 11:21:23 +08:00
Dilip Biswal 5fd62ca65a [SPARK-26215][SQL][FOLLOW-UP][MINOR] Fix the warning from ANTR4
## What changes were proposed in this pull request?
I see the following new warning from ANTR4 after SPARK-26215 after it added `SCHEMA` keyword in the reserved/unreserved list. This is a minor PR to cleanup the warning.

```
WARNING] warning(125): org/apache/spark/sql/catalyst/parser/SqlBase.g4:784:90: implicit definition of token SCHEMA in parser
[WARNING] .../apache/spark/org/apache/spark/sql/catalyst/parser/SqlBase.g4 [784:90]: implicit definition of token SCHEMA in parser
```
## How was this patch tested?
Manually built catalyst after the fix to verify

Closes #23897 from dilipbiswal/minor_parser_token.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-01 12:34:15 -08:00
liuxian 02bbe977ab [MINOR] Remove unnecessary gets when getting a value from map.
## What changes were proposed in this pull request?

Redundant `get`  when getting a value from `Map` given a key.

## How was this patch tested?

N/A

Closes #23901 from 10110346/removegetfrommap.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-01 11:48:07 -06:00
Maxim Gekk 8e5f9995ca [SPARK-27008][SQL] Support java.time.LocalDate as an external type of DateType
## What changes were proposed in this pull request?

In the PR, I propose to add new Catalyst type converter for `DateType`. It should be able to convert `java.time.LocalDate` to/from `DateType`.

Main motivations for the changes:
- Smoothly support Java 8 time API
- Avoid inconsistency of calendars used inside of Spark 3.0 (Proleptic Gregorian calendar) and `java.sql.Date` (hybrid calendar - Julian + Gregorian).
- Make conversion independent from current system timezone.

By default, Spark converts values of `DateType` to `java.sql.Date` instances but the SQL config `spark.sql.datetime.java8API.enabled` can change the behavior. If it is set to `true`, Spark uses `java.time.LocalDate` as external type for `DateType`.

## How was this patch tested?

Added new testes to `CatalystTypeConvertersSuite` to check conversion of `DateType` to/from `java.time.LocalDate`, `JavaUDFSuite`/ `UDFSuite` to test usage of `LocalDate` type in Scala/Java UDFs.

Closes #23913 from MaxGekk/date-localdate.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-01 11:04:28 +08:00
Gabor Somogyi c4bbfd177b [SPARK-24063][SS] Add maximum epoch queue threshold for ContinuousExecution
## What changes were proposed in this pull request?

Continuous processing is waiting on epochs which are not yet complete (for example one partition is not making progress) and stores pending items in queues. These queues are unbounded and can consume up all the memory easily. In this PR I've added `spark.sql.streaming.continuous.epochBacklogQueueSize` configuration possibility to make them bounded. If the related threshold reached then the query will stop with `IllegalStateException`.

## How was this patch tested?

Existing + additional unit tests.

Closes #23156 from gaborgsomogyi/SPARK-24063.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-27 09:52:43 -08:00
liuxian 7912dbb88f [MINOR] Simplify boolean expression
## What changes were proposed in this pull request?

Comparing whether Boolean expression is equal to true is redundant
For example:
The datatype of `a` is boolean.
Before:
if (a == true)
After:
if (a)

## How was this patch tested?
N/A

Closes #23884 from 10110346/simplifyboolean.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-27 08:38:00 -06:00
Maxim Gekk b0450d07bd [SPARK-26902][SQL] Support java.time.Instant as an external type of TimestampType
## What changes were proposed in this pull request?

In the PR, I propose to add new Catalyst type converter for `TimestampType`. It should be able to convert `java.time.Instant` to/from `TimestampType`.

Main motivations for the changes:
- Smoothly support Java 8 time API
- Avoid inconsistency of calendars used inside of Spark 3.0 (Proleptic Gregorian calendar) and `java.sql.Timestamp` (hybrid calendar - Julian + Gregorian).
- Make conversion independent from current system timezone.

By default, Spark converts values of `TimestampType` to `java.sql.Timestamp` instances but the SQL config `spark.sql.catalyst.timestampType` can change the behavior. It accepts two values `Timestamp` (default) and `Instant`. If the former one is set, Spark returns `java.time.Instant` instances for timestamp values.

## How was this patch tested?

Added new testes to `CatalystTypeConvertersSuite` to check conversion of `TimestampType` to/from `java.time.Instant`.

Closes #23811 from MaxGekk/timestamp-instant.

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-02-27 21:05:19 +08:00
Jungtaek Lim (HeartSaVioR) dea18ee85b [SPARK-22000][SQL] Address missing Upcast in JavaTypeInference.deserializerFor
## What changes were proposed in this pull request?

Spark expects the type of column and the type of matching field is same when deserializing to Object, but Spark hasn't actually restrict it (at least for Java bean encoder) and some users just do it and experience undefined behavior (in SPARK-22000, Spark throws compilation failure on generated code because it calls `.toString()` against primitive type.

It doesn't produce error in Scala side because `ScalaReflection.deserializerFor` properly inject Upcast if necessary. This patch proposes applying same thing to `JavaTypeInference.deserializerFor` as well.

Credit to srowen, maropu, and cloud-fan since they provided various approaches to solve this.

## How was this patch tested?

Added UT which query is slightly modified based on sample code in attachment on JIRA issue.

Closes #23854 from HeartSaVioR/SPARK-22000.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-27 13:47:20 +08:00
Hyukjin Kwon 88bc481b9e [SPARK-26830][SQL][R] Vectorized R dapply() implementation
## What changes were proposed in this pull request?

This PR targets to add vectorized `dapply()` in R, Arrow optimization.

This can be tested as below:

```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
df <- createDataFrame(mtcars)
collect(dapply(df, function(rdf) { data.frame(rdf$gear + 1) }, structType("gear double")))
```

### Requirements
  - R 3.5.x
  - Arrow package 0.12+
    ```bash
    Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
    ```

**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.

### Benchmarks

**Shall**

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false --driver-memory 4g
```

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true --driver-memory 4g
```

**R code**

```r
rdf <- read.csv("500000.csv")
df <- cache(createDataFrame(rdf))
count(df)

test <- function() {
  options(digits.secs = 6) # milliseconds
  start.time <- Sys.time()
  count(cache(dapply(df, function(rdf) { rdf }, schema(df))))
  end.time <- Sys.time()
  time.taken <- end.time - start.time
  print(time.taken)
}

test()
```

**Data (350 MB):**

```r
object.size(read.csv("500000.csv"))
350379504 bytes
```

"500000 Records"  http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/

**Results**

```
Time difference of 13.42037 mins
```

```
Time difference of 30.64156 secs
```

The performance improvement was around **2627%**.

### Limitations

- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.

- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.

## How was this patch tested?

Unit tests were added, and manually tested.

Closes #23787 from HyukjinKwon/SPARK-26830-1.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-27 14:29:58 +09:00
Liang-Chi Hsieh 0f2c0b53e8 [SPARK-26837][SQL] Pruning nested fields from object serializers
## What changes were proposed in this pull request?

In SPARK-26619, we make change to prune unnecessary individual serializers when serializing objects. This is extension to SPARK-26619. We can further prune nested fields from object serializers if they are not used.

For example, in following query, we only use one field in a struct column:

```scala
val data = Seq((("a", 1), 1), (("b", 2), 2), (("c", 3), 3))
val df = data.toDS().map(t => (t._1, t._2 + 1)).select("_1._1")
```

So, instead of having a serializer to create a two fields struct, we can prune unnecessary field from it. This is what this PR proposes to do.

In order to make this change conservative and safer, a SQL config is added to control it. It is disabled by default.

TODO: Support to prune nested fields inside MapType's key and value.

## How was this patch tested?

Added tests.

Closes #23740 from viirya/nested-pruning-serializer-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-27 12:45:24 +08:00
Maxim Gekk a2a41b7bf2 [SPARK-26978][CORE][SQL] Avoid magic time constants
## What changes were proposed in this pull request?

In the PR, I propose to refactor existing code related to date/time conversions, and replace constants like `1000` and `1000000` by `DateTimeUtils` constants and transformation functions from `java.util.concurrent.TimeUnit._`.

## How was this patch tested?

The changes are tested by existing test suites.

Closes #23878 from MaxGekk/magic-time-constants.

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-02-26 09:08:12 -06:00
Maxim Gekk 75c48ac36d [SPARK-26908][SQL] Fix DateTimeUtils.toMillis and millisToDays
## What changes were proposed in this pull request?

The `DateTimeUtils.toMillis` can produce inaccurate result for some negative values (timestamps before epoch). The error can be around 1ms. In the PR, I propose to use `Math.floorDiv` in casting microseconds to milliseconds, and milliseconds to days since epoch.

## How was this patch tested?

Added new test to `DateTimeUtilsSuite`, and tested by `CastSuite` as well.

Closes #23815 from MaxGekk/micros-to-millis.

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-02-23 11:35:11 -06:00