Commit graph

9713 commits

Author SHA1 Message Date
Dongjoon Hyun aed8dbab1d [SPARK-32364][SQL][FOLLOWUP] Add toMap to return originalMap and documentation
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/29160. We already removed the indeterministicity. This PR aims the following for the existing code base.
1. Add an explicit document to `DataFrameReader/DataFrameWriter`.

2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`.

3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`.
```scala
- val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
+ val params = extraOptions ++ connectionProperties.asScala
```

4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later.
```scala
- val options = sessionOptions ++ extraOptions
+ val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap
  val dsOptions = new CaseInsensitiveStringMap(options.asJava)
```

### Why are the changes needed?

`extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`.

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

No.

### How was this patch tested?

Pass the Jenkins or GitHub Action with the existing tests and newly add test case at `JDBCSuite`.

Closes #29191 from dongjoon-hyun/SPARK-32364-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 06:28:08 -07:00
LantaoJin 182566bf57 [SPARK-32237][SQL] Resolve hint in CTE
### What changes were proposed in this pull request?
This PR is to move `Substitution` rule before `Hints` rule in `Analyzer` to avoid hint in CTE not working.

### Why are the changes needed?
Below SQL in Spark3.0 will throw AnalysisException, but it works in Spark2.x
```sql
WITH cte AS (SELECT /*+ REPARTITION(3) */ T.id, T.data FROM $t1 T)
SELECT cte.id, cte.data FROM cte
```
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`cte.id`' given input columns: [cte.data, cte.id]; line 3 pos 7;
'Project ['cte.id, 'cte.data]
+- SubqueryAlias cte
   +- Project [id#21L, data#22]
      +- SubqueryAlias T
         +- SubqueryAlias testcat.ns1.ns2.tbl
            +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl

'Project ['cte.id, 'cte.data]
+- SubqueryAlias cte
   +- Project [id#21L, data#22]
      +- SubqueryAlias T
         +- SubqueryAlias testcat.ns1.ns2.tbl
            +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl
```

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

### How was this patch tested?
Add a unit test

Closes #29062 from LantaoJin/SPARK-32237.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 03:10:45 +00:00
Takuya UESHIN 46169823c0 [SPARK-30616][SQL][FOLLOW-UP] Use only config key name in the config doc
### What changes were proposed in this pull request?

This is a follow-up of #28852.

This PR to use only config name; otherwise the doc for the config entry shows the entire details of the referring configs.

### Why are the changes needed?

The doc for the newly introduced config entry shows the entire details of the referring configs.

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

The doc for the config entry will show only the referring config keys.

### How was this patch tested?

Existing tests.

Closes #29194 from ueshin/issues/SPARK-30616/fup.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 03:07:30 +00:00
Kent Yao b151194299 [SPARK-32392][SQL] Reduce duplicate error log for executing sql statement operation in thrift server
### What changes were proposed in this pull request?

This PR removes the duplicated error log which has been logged in `org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation#execute` but logged again in `runInternal`.

Besides, the log4j configuration for SparkExecuteStatementOperation is turned off because it's not very friendly for Jenkins

### Why are the changes needed?

remove the duplicated error log for better  user experience

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

Yes, less log in thrift server's driver log

### How was this patch tested?

locally verified the result in target/unit-test.log

Closes #29189 from yaooqinn/SPARK-32392.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-23 10:12:13 +09:00
ulysses 184074de22 [SPARK-31999][SQL] Add REFRESH FUNCTION command
### What changes were proposed in this pull request?

In Hive mode, permanent functions are shared with Hive metastore so that functions may be modified by other Hive client. With in long-lived spark scene, it's hard to update the change of function.

Here are 2 reasons:
* Spark cache the function in memory using `FunctionRegistry`.
* User may not know the location or classname of udf when using `replace function`.

Note that we use v2 command code path to add new command.

### Why are the changes needed?

Give a easy way to make spark function registry sync with Hive metastore.
Then we can call
```
refresh function functionName
```

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

Yes, new command.

### How was this patch tested?

New UT.

Closes #28840 from ulysses-you/SPARK-31999.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 19:05:50 +00:00
Dongjoon Hyun cd16a10475 [SPARK-32364][SQL] Use CaseInsensitiveMap for DataFrameReader/Writer options
### What changes were proposed in this pull request?

When a user have multiple options like `path`, `paTH`, and `PATH` for the same key `path`, `option/options` is non-deterministic because `extraOptions` is `HashMap`. This PR aims to use `CaseInsensitiveMap` instead of `HashMap` to fix this bug fundamentally.

### Why are the changes needed?

Like the following, DataFrame's `option/options` have been non-deterministic in terms of case-insensitivity because it stores the options at `extraOptions` which is using `HashMap` class.

```scala
spark.read
  .option("paTh", "1")
  .option("PATH", "2")
  .option("Path", "3")
  .option("patH", "4")
  .load("5")
...
org.apache.spark.sql.AnalysisException:
Path does not exist: file:/.../1;
```

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

Yes. However, this is a bug fix for the indeterministic cases.

### How was this patch tested?

Pass the Jenkins or GitHub Action with newly added test cases.

Closes #29160 from dongjoon-hyun/SPARK-32364.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-22 07:58:45 -07:00
Takeshi Yamamuro 04bf3511f1 [SPARK-21117][SQL][FOLLOWUP] Define prettyName for WidthBucket
### What changes were proposed in this pull request?

This PR is to define prettyName for `WidthBucket`.
This comes from the gatorsmile's suggestion: https://github.com/apache/spark/pull/28764#discussion_r457802957

### Why are the changes needed?

For a better name.

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

No.

### How was this patch tested?

Existing tests.

Closes #29183 from maropu/SPARK-21117-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-22 02:51:30 -07:00
Max Gekk feca9edbdd [MINOR][SQL][TESTS] Create tables once in JDBC tests
### What changes were proposed in this pull request?
In PR, I propose to create input tables once before executing tests in `JDBCSuite` and `JdbcRDDSuite`. Currently, the table are created before every test in the test suites.

### Why are the changes needed?
This speed up the test suites up 30-40%.

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

### How was this patch tested?
Run the modified test suites

Closes #29176 from MaxGekk/jdbc-suite-before-all.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 08:32:01 +00:00
Kent Yao 29b7eaa438 [MINOR][SQL] Fix warning message for ThriftCLIService.GetCrossReference and GetPrimaryKeys
### What changes were proposed in this pull request?

This PR fixes the warning message for ThriftCLIService.GetCrossReference and GetPrimaryKeys

### Why are the changes needed?

Although we haven't had our own implementation for these thrift APIs, but it still worth logging the right message when people call them wrongly.

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

yes, the driver log for the thrift server will log the right message for the ThriftCLIService.GetCrossReference and GetPrimaryKeys APIs

### How was this patch tested?

passing Jenkins.

Closes #29184 from yaooqinn/minor.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 08:15:04 +00:00
Cheng Su 39181ff209 [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable
### What changes were proposed in this pull request?
Based on a follow up comment in https://github.com/apache/spark/pull/28123, where we can coalesce buckets for shuffled hash join as well. The note here is we only coalesce the buckets from shuffled hash join stream side (i.e. the side not building hash map), so we don't need to worry about OOM when coalescing multiple buckets in one task for building hash map.

> If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.

Refactor existing physical plan rule `CoalesceBucketsInSortMergeJoin` to `CoalesceBucketsInJoin`, for covering shuffled hash join as well.
Refactor existing unit test `CoalesceBucketsInSortMergeJoinSuite` to `CoalesceBucketsInJoinSuite`, for covering shuffled hash join as well.

### Why are the changes needed?
Avoid shuffle for joining different bucketed tables, is also useful for shuffled hash join. In production, we are seeing users to use shuffled hash join to join bucketed tables (set `spark.sql.join.preferSortMergeJoin`=false, to avoid sort), and this can help avoid shuffle if number of buckets are not same.

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

### How was this patch tested?
Added unit tests in `CoalesceBucketsInJoinSuite` for verifying shuffled hash join physical plan.

### Performance number per request from maropu

I was looking at TPCDS per suggestion from maropu. But I found most of queries from TPCDS are doing aggregate, and only several ones are doing join. None of input tables are bucketed. So I took the approach to test a modified version of `TPCDS q93` as

```
SELECT ss_ticket_number, sr_ticket_number
FROM store_sales
JOIN store_returns
ON ss_ticket_number = sr_ticket_number
```

And make `store_sales` and `store_returns` to be bucketed tables.

Physical query plan without coalesce:

```
ShuffledHashJoin [ss_ticket_number#109L], [sr_ticket_number#120L], Inner, BuildLeft
:- Exchange hashpartitioning(ss_ticket_number#109L, 4), true, [id=#67]
:  +- *(1) Project [ss_ticket_number#109L]
:     +- *(1) Filter isnotnull(ss_ticket_number#109L)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.store_sales[ss_ticket_number#109L] Batched: true, DataFilters: [isnotnull(ss_ticket_number#109L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_sales], PartitionFilters: [], PushedFilters: [IsNotNull(ss_ticket_number)], ReadSchema: struct<ss_ticket_number:bigint>, SelectedBucketsCount: 2 out of 2
+- *(2) Project [sr_returned_date_sk#111L, sr_return_time_sk#112L, sr_item_sk#113L, sr_customer_sk#114L, sr_cdemo_sk#115L, sr_hdemo_sk#116L, sr_addr_sk#117L, sr_store_sk#118L, sr_reason_sk#119L, sr_ticket_number#120L, sr_return_quantity#121L, sr_return_amt#122, sr_return_tax#123, sr_return_amt_inc_tax#124, sr_fee#125, sr_return_ship_cost#126, sr_refunded_cash#127, sr_reversed_charge#128, sr_store_credit#129, sr_net_loss#130]
   +- *(2) Filter isnotnull(sr_ticket_number#120L)
      +- *(2) ColumnarToRow
         +- FileScan parquet default.store_returns[sr_returned_date_sk#111L,sr_return_time_sk#112L,sr_item_sk#113L,sr_customer_sk#114L,sr_cdemo_sk#115L,sr_hdemo_sk#116L,sr_addr_sk#117L,sr_store_sk#118L,sr_reason_sk#119L,sr_ticket_number#120L,sr_return_quantity#121L,sr_return_amt#122,sr_return_tax#123,sr_return_amt_inc_tax#124,sr_fee#125,sr_return_ship_cost#126,sr_refunded_cash#127,sr_reversed_charge#128,sr_store_credit#129,sr_net_loss#130] Batched: true, DataFilters: [isnotnull(sr_ticket_number#120L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_ticket_number)], ReadSchema: struct<sr_returned_date_sk:bigint,sr_return_time_sk:bigint,sr_item_sk:bigint,sr_customer_sk:bigin..., SelectedBucketsCount: 4 out of 4
```

Physical query plan with coalesce:

```
ShuffledHashJoin [ss_ticket_number#109L], [sr_ticket_number#120L], Inner, BuildLeft
:- *(1) Project [ss_ticket_number#109L]
:  +- *(1) Filter isnotnull(ss_ticket_number#109L)
:     +- *(1) ColumnarToRow
:        +- FileScan parquet default.store_sales[ss_ticket_number#109L] Batched: true, DataFilters: [isnotnull(ss_ticket_number#109L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_sales], PartitionFilters: [], PushedFilters: [IsNotNull(ss_ticket_number)], ReadSchema: struct<ss_ticket_number:bigint>, SelectedBucketsCount: 2 out of 2
+- *(2) Project [sr_returned_date_sk#111L, sr_return_time_sk#112L, sr_item_sk#113L, sr_customer_sk#114L, sr_cdemo_sk#115L, sr_hdemo_sk#116L, sr_addr_sk#117L, sr_store_sk#118L, sr_reason_sk#119L, sr_ticket_number#120L, sr_return_quantity#121L, sr_return_amt#122, sr_return_tax#123, sr_return_amt_inc_tax#124, sr_fee#125, sr_return_ship_cost#126, sr_refunded_cash#127, sr_reversed_charge#128, sr_store_credit#129, sr_net_loss#130]
   +- *(2) Filter isnotnull(sr_ticket_number#120L)
      +- *(2) ColumnarToRow
         +- FileScan parquet default.store_returns[sr_returned_date_sk#111L,sr_return_time_sk#112L,sr_item_sk#113L,sr_customer_sk#114L,sr_cdemo_sk#115L,sr_hdemo_sk#116L,sr_addr_sk#117L,sr_store_sk#118L,sr_reason_sk#119L,sr_ticket_number#120L,sr_return_quantity#121L,sr_return_amt#122,sr_return_tax#123,sr_return_amt_inc_tax#124,sr_fee#125,sr_return_ship_cost#126,sr_refunded_cash#127,sr_reversed_charge#128,sr_store_credit#129,sr_net_loss#130] Batched: true, DataFilters: [isnotnull(sr_ticket_number#120L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_ticket_number)], ReadSchema: struct<sr_returned_date_sk:bigint,sr_return_time_sk:bigint,sr_item_sk:bigint,sr_customer_sk:bigin..., SelectedBucketsCount: 4 out of 4 (Coalesced to 2)
```

Run time improvement as 50% of wall clock time:

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join coalesce bucket off              1541           1664         106          1.9         535.1       1.0X
shuffle hash join coalesce bucket on               1060           1169          81          2.7         368.1       1.5X
```

Closes #29079 from c21/split-bucket.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-22 08:56:26 +09:00
Dongjoon Hyun 8c7d6f9733 [SPARK-32377][SQL] CaseInsensitiveMap should be deterministic for addition
### What changes were proposed in this pull request?

This PR aims to fix `CaseInsensitiveMap` to be deterministic for addition.

### Why are the changes needed?

```scala
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
var m = CaseInsensitiveMap(Map.empty[String, String])
Seq(("paTh", "1"), ("PATH", "2"), ("Path", "3"), ("patH", "4"), ("path", "5")).foreach { kv =>
  m = (m + kv).asInstanceOf[CaseInsensitiveMap[String]]
  println(m.get("path"))
}
```

**BEFORE**
```
Some(1)
Some(2)
Some(3)
Some(4)
Some(1)
```

**AFTER**
```
Some(1)
Some(2)
Some(3)
Some(4)
Some(5)
```

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

Yes, but this is a bug fix on non-deterministic behavior.

### How was this patch tested?

Pass the newly added test case.

Closes #29172 from dongjoon-hyun/SPARK-32377.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 22:20:16 -07:00
LantaoJin 8a1c24bb03 [SPARK-32362][SQL][TEST] AdaptiveQueryExecSuite misses verifying AE results
### What changes were proposed in this pull request?
Verify results for `AdaptiveQueryExecSuite`

### Why are the changes needed?
`AdaptiveQueryExecSuite` misses verifying AE results
```scala
QueryTest.sameRows(result.toSeq, df.collect().toSeq)
```
Even the results are different, no fail.

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

### How was this patch tested?
Exists unit tests.

Closes #29158 from LantaoJin/SPARK-32362.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-21 12:47:45 +09:00
gengjiaan 02114f96d6 [SPARK-32365][SQL] Add a boundary condition for negative index in regexp_extract
### What changes were proposed in this pull request?
The current implement of regexp_extract will throws a unprocessed exception show below:
SELECT regexp_extract('1a 2b 14m', 'd+' -1)

```
java.lang.IndexOutOfBoundsException: No group -1
java.util.regex.Matcher.group(Matcher.java:538)
org.apache.spark.sql.catalyst.expressions.RegExpExtract.nullSafeEval(regexpExpressions.scala:455)
org.apache.spark.sql.catalyst.expressions.TernaryExpression.eval(Expression.scala:704)
org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
```

### Why are the changes needed?
Fix a bug `java.lang.IndexOutOfBoundsException: No group -1`

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

### How was this patch tested?
new UT

Closes #29161 from beliefer/regexp_extract-group-not-allow-less-than-zero.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 20:34:51 -07:00
Nik Vanderhoof 7d65caebec [SPARK-32338][SQL] Overload slice to accept Column for start and length
### What changes were proposed in this pull request?

Add an overload for the `slice` function that can accept Columns for the `start` and `length` parameters.

### Why are the changes needed?

This will allow users to take slices of arrays based on the length of the arrays, or via data in other columns.
```scala
df.select(slice(x, 4, size(x) - 4))
```

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

Yes, before the `slice` method would only accept Ints for the start and length parameters, now we can pass in Columns and/or Ints.

### How was this patch tested?

I've extended the existing tests for slice but using combinations of Column and Ints.

Closes #29138 from nvander1/SPARK-32338.

Authored-by: Nik Vanderhoof <nikolasrvanderhoof@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2020-07-20 17:48:07 -07:00
HyukjinKwon 133c5edc80 [SPARK-32368][SQL] pathGlobFilter, recursiveFileLookup and basePath should respect case insensitivity
### What changes were proposed in this pull request?

This PR proposes to make the datasource options at `PartitioningAwareFileIndex` respect case insensitivity consistently:
- `pathGlobFilter`
- `recursiveFileLookup `
- `basePath`

### Why are the changes needed?

To support consistent case insensitivity in datasource options.

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

Yes, now users can also use case insensitive options such as `PathglobFilter`.

### How was this patch tested?

Unittest were added. It reuses existing tests and adds extra clues to make it easier to track when the test is broken.

Closes #29165 from HyukjinKwon/SPARK-32368.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 13:56:00 -07:00
Cheng Su fe07521c9e [SPARK-32330][SQL] Preserve shuffled hash join build side partitioning
### What changes were proposed in this pull request?

Currently `ShuffledHashJoin.outputPartitioning` inherits from `HashJoin.outputPartitioning`, which only preserves stream side partitioning (`HashJoin.scala`):

```
override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning
```

This loses build side partitioning information, and causes extra shuffle if there's another join / group-by after this join.

Example:

```
withSQLConf(
    SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50",
    SQLConf.SHUFFLE_PARTITIONS.key -> "2",
    SQLConf.PREFER_SORTMERGEJOIN.key -> "false") {
  val df1 = spark.range(10).select($"id".as("k1"))
  val df2 = spark.range(30).select($"id".as("k2"))
  Seq("inner", "cross").foreach(joinType => {
    val plan = df1.join(df2, $"k1" === $"k2", joinType).groupBy($"k1").count()
      .queryExecution.executedPlan
    assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1)
    // No extra shuffle before aggregate
    assert(plan.collect { case _: ShuffleExchangeExec => true }.size === 2)
  })
}
```

Current physical plan (having an extra shuffle on `k1` before aggregate)

```
*(4) HashAggregate(keys=[k1#220L], functions=[count(1)], output=[k1#220L, count#235L])
+- Exchange hashpartitioning(k1#220L, 2), true, [id=#117]
   +- *(3) HashAggregate(keys=[k1#220L], functions=[partial_count(1)], output=[k1#220L, count#239L])
      +- *(3) Project [k1#220L]
         +- ShuffledHashJoin [k1#220L], [k2#224L], Inner, BuildLeft
            :- Exchange hashpartitioning(k1#220L, 2), true, [id=#109]
            :  +- *(1) Project [id#218L AS k1#220L]
            :     +- *(1) Range (0, 10, step=1, splits=2)
            +- Exchange hashpartitioning(k2#224L, 2), true, [id=#111]
               +- *(2) Project [id#222L AS k2#224L]
                  +- *(2) Range (0, 30, step=1, splits=2)
```

Ideal physical plan (no shuffle on `k1` before aggregate)

```
*(3) HashAggregate(keys=[k1#220L], functions=[count(1)], output=[k1#220L, count#235L])
+- *(3) HashAggregate(keys=[k1#220L], functions=[partial_count(1)], output=[k1#220L, count#239L])
   +- *(3) Project [k1#220L]
      +- ShuffledHashJoin [k1#220L], [k2#224L], Inner, BuildLeft
         :- Exchange hashpartitioning(k1#220L, 2), true, [id=#107]
         :  +- *(1) Project [id#218L AS k1#220L]
         :     +- *(1) Range (0, 10, step=1, splits=2)
         +- Exchange hashpartitioning(k2#224L, 2), true, [id=#109]
            +- *(2) Project [id#222L AS k2#224L]
               +- *(2) Range (0, 30, step=1, splits=2)
```

This can be fixed by overriding `outputPartitioning` method in `ShuffledHashJoinExec`, similar to `SortMergeJoinExec`.
In addition, also fix one typo in `HashJoin`, as that code path is shared between broadcast hash join and shuffled hash join.

### Why are the changes needed?

To avoid shuffle (for queries having multiple joins or group-by), for saving CPU and IO.

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

No.

### How was this patch tested?

Added unit test in `JoinSuite`.

Closes #29130 from c21/shj.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-20 14:38:43 +00:00
Terry Kim e0ecb66f53 [SPARK-31869][SQL] BroadcastHashJoinExec can utilize the build side for its output partitioning
### What changes were proposed in this pull request?

Currently, the `BroadcastHashJoinExec`'s `outputPartitioning` only uses the streamed side's `outputPartitioning`. However, if the join type of `BroadcastHashJoinExec` is an inner-like join, the build side's info (the join keys) can be added to `BroadcastHashJoinExec`'s `outputPartitioning`.

 For example,
```Scala
spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "500")
val t1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1")
val t2 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i2", "j2")
val t3 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i3", "j3")
val t4 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i4", "j4")

// join1 is a sort merge join.
val join1 = t1.join(t2, t1("i1") === t2("i2"))

// join2 is a broadcast join where t3 is broadcasted.
val join2 = join1.join(t3, join1("i1") === t3("i3"))

// Join on the column from the broadcasted side (i3).
val join3 = join2.join(t4, join2("i3") === t4("i4"))

join3.explain
```
You see that `Exchange hashpartitioning(i2#103, 200)` is introduced because there is no output partitioning info from the build side.
```
== Physical Plan ==
*(6) SortMergeJoin [i3#29], [i4#40], Inner
:- *(4) Sort [i3#29 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(i3#29, 200), true, [id=#55]
:     +- *(3) BroadcastHashJoin [i1#7], [i3#29], Inner, BuildRight
:        :- *(3) SortMergeJoin [i1#7], [i2#18], Inner
:        :  :- *(1) Sort [i1#7 ASC NULLS FIRST], false, 0
:        :  :  +- Exchange hashpartitioning(i1#7, 200), true, [id=#28]
:        :  :     +- LocalTableScan [i1#7, j1#8]
:        :  +- *(2) Sort [i2#18 ASC NULLS FIRST], false, 0
:        :     +- Exchange hashpartitioning(i2#18, 200), true, [id=#29]
:        :        +- LocalTableScan [i2#18, j2#19]
:        +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#34]
:           +- LocalTableScan [i3#29, j3#30]
+- *(5) Sort [i4#40 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i4#40, 200), true, [id=#39]
      +- LocalTableScan [i4#40, j4#41]
```
This PR proposes to introduce output partitioning for the build side for `BroadcastHashJoinExec` if the streamed side has a `HashPartitioning` or a collection of `HashPartitioning`s.

There is a new internal config `spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit`, which can limit the number of partitioning a `HashPartitioning` can expand to. It can be set to "0" to disable this feature.

### Why are the changes needed?

To remove unnecessary shuffle.

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

Yes, now the shuffle in the above example can be eliminated:
```
== Physical Plan ==
*(5) SortMergeJoin [i3#108], [i4#119], Inner
:- *(3) Sort [i3#108 ASC NULLS FIRST], false, 0
:  +- *(3) BroadcastHashJoin [i1#86], [i3#108], Inner, BuildRight
:     :- *(3) SortMergeJoin [i1#86], [i2#97], Inner
:     :  :- *(1) Sort [i1#86 ASC NULLS FIRST], false, 0
:     :  :  +- Exchange hashpartitioning(i1#86, 200), true, [id=#120]
:     :  :     +- LocalTableScan [i1#86, j1#87]
:     :  +- *(2) Sort [i2#97 ASC NULLS FIRST], false, 0
:     :     +- Exchange hashpartitioning(i2#97, 200), true, [id=#121]
:     :        +- LocalTableScan [i2#97, j2#98]
:     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#126]
:        +- LocalTableScan [i3#108, j3#109]
+- *(4) Sort [i4#119 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i4#119, 200), true, [id=#130]
      +- LocalTableScan [i4#119, j4#120]
```

### How was this patch tested?

Added new tests.

Closes #28676 from imback82/broadcast_join_output.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-20 14:25:51 +00:00
Gengliang Wang d0c83f372b [SPARK-32302][SQL] Partially push down disjunctive predicates through Join/Partitions
### What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/28733 and #28805, CNF conversion is used to push down disjunctive predicates through join and partitions pruning.

It's a good improvement, however, converting all the predicates in CNF can lead to a very long result, even with grouping functions over expressions.  For example, for the following predicate
```
(p0 = '1' AND p1 = '1') OR (p0 = '2' AND p1 = '2') OR (p0 = '3' AND p1 = '3') OR (p0 = '4' AND p1 = '4') OR (p0 = '5' AND p1 = '5') OR (p0 = '6' AND p1 = '6') OR (p0 = '7' AND p1 = '7') OR (p0 = '8' AND p1 = '8') OR (p0 = '9' AND p1 = '9') OR (p0 = '10' AND p1 = '10') OR (p0 = '11' AND p1 = '11') OR (p0 = '12' AND p1 = '12') OR (p0 = '13' AND p1 = '13') OR (p0 = '14' AND p1 = '14') OR (p0 = '15' AND p1 = '15') OR (p0 = '16' AND p1 = '16') OR (p0 = '17' AND p1 = '17') OR (p0 = '18' AND p1 = '18') OR (p0 = '19' AND p1 = '19') OR (p0 = '20' AND p1 = '20')
```
will be converted into a long query(130K characters) in Hive metastore, and there will be error:
```
javax.jdo.JDOException: Exception thrown when executing query : SELECT DISTINCT 'org.apache.hadoop.hive.metastore.model.MPartition' AS NUCLEUS_TYPE,A0.CREATE_TIME,A0.LAST_ACCESS_TIME,A0.PART_NAME,A0.PART_ID,A0.PART_NAME AS NUCORDER0 FROM PARTITIONS A0 LEFT OUTER JOIN TBLS B0 ON A0.TBL_ID = B0.TBL_ID LEFT OUTER JOIN DBS C0 ON B0.DB_ID = C0.DB_ID WHERE B0.TBL_NAME = ? AND C0."NAME" = ? AND ((((((A0.PART_NAME LIKE '%/p1=1' ESCAPE '\' ) OR (A0.PART_NAME LIKE '%/p1=2' ESCAPE '\' )) OR (A0.PART_NAME LIKE '%/p1=3' ESCAPE '\' )) OR ((A0.PART_NAME LIKE '%/p1=4' ESCAPE '\' ) O ...
```

Essentially, we just need to traverse predicate and extract the convertible sub-predicates like what we did in https://github.com/apache/spark/pull/24598. There is no need to maintain the CNF result set.

### Why are the changes needed?

A better implementation for pushing down disjunctive and complex predicates. The pushed down predicates is always equal or shorter than the CNF result.

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

No

### How was this patch tested?

Unit tests

Closes #29101 from gengliangwang/pushJoin.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-20 14:17:31 +00:00
Anton Okolnychyi 0aca1a6ed4 [SPARK-32276][SQL] Remove redundant sorts before repartition nodes
### What changes were proposed in this pull request?

This PR proposes to remove redundant sorts before repartition nodes whenever the data is ordered after the repartitioning.

### Why are the changes needed?

It looks like our `EliminateSorts` rule can be extended further to remove sorts before repartition nodes that don't affect the final output ordering. It seems safe to perform the following rewrites:

- `Sort -> Repartition -> Sort -> Scan` as `Sort -> Repartition -> Scan`
- `Sort -> Repartition -> Project -> Sort -> Scan` as `Sort -> Repartition -> Project -> Scan`

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

No.

### How was this patch tested?

More test cases.

Closes #29089 from aokolnychyi/spark-32276.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-19 12:11:26 -07:00
Takeshi Yamamuro c7a68a920d [SPARK-32344][SQL] Unevaluable expr is set to FIRST/LAST ignoreNullsExpr in distinct aggregates
### What changes were proposed in this pull request?

This PR intends to fix a bug of distinct FIRST/LAST aggregates in v2.4.6/v3.0.0/master;
```
scala> sql("SELECT FIRST(DISTINCT v) FROM VALUES 1, 2, 3 t(v)").show()
...
Caused by: java.lang.UnsupportedOperationException: Cannot evaluate expression: false#37
  at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.eval(Expression.scala:258)
  at org.apache.spark.sql.catalyst.expressions.AttributeReference.eval(namedExpressions.scala:226)
  at org.apache.spark.sql.catalyst.expressions.aggregate.First.ignoreNulls(First.scala:68)
  at org.apache.spark.sql.catalyst.expressions.aggregate.First.updateExpressions$lzycompute(First.scala:82)
  at org.apache.spark.sql.catalyst.expressions.aggregate.First.updateExpressions(First.scala:81)
  at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$15.apply(HashAggregateExec.scala:268)
```
A root cause of this bug is that the `Aggregation` strategy replaces a foldable boolean `ignoreNullsExpr` expr with a `Unevaluable` expr (`AttributeReference`) for distinct FIRST/LAST aggregate functions. But, this operation cannot be allowed because the `Analyzer` has checked that it must be foldabe;
ffdbbae1d4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala (L74-L76)
So, this PR proposes to change a vriable for `IGNORE NULLS`  from `Expression` to `Boolean` to avoid the case.

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Added a test in `DataFrameAggregateSuite`.

Closes #29143 from maropu/SPARK-32344.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-19 11:11:42 +09:00
Sean Owen ee624821a9 [SPARK-29292][YARN][K8S][MESOS] Fix Scala 2.13 compilation for remaining modules
### What changes were proposed in this pull request?

See again the related PRs like https://github.com/apache/spark/pull/28971
This completes fixing compilation for 2.13 for all but `repl`, which is a separate task.

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

No

### How was this patch tested?

Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12)

Closes #29147 from srowen/SPARK-29292.4.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-18 15:08:00 -07:00
Sudharshann D f9f9309bec [SPARK-31579][SQL] replaced floorDiv to Div
### What changes were proposed in this pull request?

Replaced  floorDiv to just / in `localRebaseGregorianToJulianDays()` in `spark/sql/catalyst/util/RebaseDateTime.scala`

### Why are the changes needed?

Easier to understand the logic/code and a little more efficiency.

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

NO

### How was this patch tested?

Proof of concept [here](https://github.com/apache/spark/pull/28573/files). The operation `utcCal.getTimeInMillis / MILLIS_PER_DAY` results in an interger value already.

Closes #29008 from Sudhar287/SPARK-31579.

Authored-by: Sudharshann D <sudhar287@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-07-18 13:04:58 -05:00
Yaroslav Tkachenko 34baed8139 [SPARK-30616][SQL] Introduce TTL config option for SQL Metadata Cache
### What changes were proposed in this pull request?
New `spark.sql.metadataCacheTTLSeconds` option that adds time-to-live cache behaviour to the existing caches in `FileStatusCache` and `SessionCatalog`.

### Why are the changes needed?
Currently Spark [caches file listing for tables](https://spark.apache.org/docs/2.4.4/sql-data-sources-parquet.html#metadata-refreshing) and requires issuing `REFRESH TABLE` any time the file listing has changed outside of Spark. Unfortunately, simply submitting `REFRESH TABLE` commands could be very cumbersome. Assuming frequently added files, hundreds of tables and dozens of users querying the data (and expecting up-to-date results), manually refreshing metadata for each table is not a solution.

This is a pretty common use-case for streaming ingestion of data, which can be done outside of Spark (with tools like Kafka Connect, etc.).

A similar feature exists in Presto: `hive.file-status-cache-expire-time` can be found [here](https://prestosql.io/docs/current/connector/hive.html#hive-configuration-properties).

### Does this PR introduce _any_ user-facing change?
Yes, it's controlled with the new `spark.sql.metadataCacheTTLSeconds` option.

When it's set to `-1` (by default), the behaviour of caches doesn't change, so it stays _backwards-compatible_.

Otherwise, you can specify a value in seconds, for example `spark.sql.metadataCacheTTLSeconds: 60` means 1-minute cache TTL.

### How was this patch tested?

Added new tests in:

- FileIndexSuite
- SessionCatalogSuite

Closes #28852 from sap1ens/SPARK-30616-metadata-cache-ttl.

Authored-by: Yaroslav Tkachenko <sapiensy@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-17 13:40:54 +00:00
Kent Yao efa70b8755 [SPARK-32145][SQL][FOLLOWUP] Fix type in the error log of SparkOperation
### What changes were proposed in this pull request?

Fix typo error in the error log of SparkOperation trait, reported by https://github.com/apache/spark/pull/28963#discussion_r454954542

### Why are the changes needed?

fix error in thrift server driver log

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

No

### How was this patch tested?

Passing GitHub actions

Closes #29140 from yaooqinn/SPARK-32145-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-17 04:50:26 +00:00
Frank Yin 9747e8fc9d [SPARK-31831][SQL][TESTS][FOLLOWUP] Put mocks for HiveSessionImplSuite in hive version related subdirectories
### What changes were proposed in this pull request?

This patch fixes the build issue on Hive 1.2 profile brought by #29069, via putting mocks for HiveSessionImplSuite in hive version related subdirectories, so that maven build will pick up the proper source code according to the profile.

### Why are the changes needed?

#29069 fixed the flakiness of HiveSessionImplSuite, but given the patch relied on the default profile (Hive 2.3) it broke the build with Hive 1.2 profile. This patch addresses both Hive versions.

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

No

### How was this patch tested?

Manually confirmed the test suite via below command:

> Hive 1.2
```
build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.HiveSessionImplSuite test -Phive-1.2 -Phadoop-2.7 -Phive-thriftserver
```

> Hive 2.3

```
build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.HiveSessionImplSuite test -Phive-2.3 -Phadoop-3.2 -Phive-thriftserver
```

Closes #29129 from frankyin-factual/hive-tests.

Authored-by: Frank Yin <frank@factual.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-17 11:14:25 +09:00
Maxim Gekk c1f160e097 [SPARK-30648][SQL] Support filters pushdown in JSON datasource
### What changes were proposed in this pull request?
In the PR, I propose to support pushed down filters in JSON datasource. The reason of pushing a filter up to `JacksonParser` is to apply the filter as soon as all its attributes become available i.e. converted from JSON field values to desired values according to the schema. This allows to skip parsing of the rest of JSON record and conversions of other values if the filter returns `false`. This can improve performance when pushed filters are highly selective and conversion of JSON string fields to desired values are comparably expensive ( for example, the conversion to `TIMESTAMP` values).

The main idea behind of `JsonFilters` is to group pushdown filters by their references, convert the grouped filters to expressions, and then compile to predicates. The predicates are indexed by schema field positions. Each predicate has a state with reference counter to non-set row fields. As soon as the counter reaches `0`, it can be applied to the row because all its dependencies has been set. Before processing new row, predicate's reference counter is reset to total number of predicate references (dependencies in a row).

The common code shared between `CSVFilters` and `JsonFilters` is moved to the `StructFilters` class and its companion object.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks up to **27 times** on JDK 8 and **25** times on JDK 11:
```
OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1044-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                       25230          25255          22          0.0      252299.6       1.0X
pushdown disabled                                 25248          25282          33          0.0      252475.6       1.0X
w/ filters                                          905            911           8          0.1        9047.9      27.9X
```

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

### How was this patch tested?
- Added new test suites `JsonFiltersSuite` and `JacksonParserSuite`.
- By new end-to-end and case sensitivity tests in `JsonSuite`.
- By `CSVFiltersSuite`, `UnivocityParserSuite` and `CSVSuite`.
- Re-running `CSVBenchmark` and `JsonBenchmark` using Amazon EC2:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`|

and `./dev/run-benchmarks`:
```python
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
    ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #27366 from MaxGekk/json-filters-pushdown.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-17 00:01:13 +09:00
SaurabhChawla 6be8b935a4 [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables
### What changes were proposed in this pull request?
Spark sql commands are failing on selecting the orc tables
Steps to reproduce
Example 1 -
Prerequisite -  This is the location(/Users/test/tpcds_scale5data/date_dim) for orc data which is generated by the hive.
```
val table = """CREATE TABLE `date_dim` (
  `d_date_sk` INT,
  `d_date_id` STRING,
  `d_date` TIMESTAMP,
  `d_month_seq` INT,
  `d_week_seq` INT,
  `d_quarter_seq` INT,
  `d_year` INT,
  `d_dow` INT,
  `d_moy` INT,
  `d_dom` INT,
  `d_qoy` INT,
  `d_fy_year` INT,
  `d_fy_quarter_seq` INT,
  `d_fy_week_seq` INT,
  `d_day_name` STRING,
  `d_quarter_name` STRING,
  `d_holiday` STRING,
  `d_weekend` STRING,
  `d_following_holiday` STRING,
  `d_first_dom` INT,
  `d_last_dom` INT,
  `d_same_day_ly` INT,
  `d_same_day_lq` INT,
  `d_current_day` STRING,
  `d_current_week` STRING,
  `d_current_month` STRING,
  `d_current_quarter` STRING,
  `d_current_year` STRING)
USING orc
LOCATION '/Users/test/tpcds_scale5data/date_dim'"""

spark.sql(table).collect

val u = """select date_dim.d_date_id from date_dim limit 5"""

spark.sql(u).collect
```
Example 2

```
  val table = """CREATE TABLE `test_orc_data` (
  `_col1` INT,
  `_col2` STRING,
  `_col3` INT)
  USING orc"""

spark.sql(table).collect

spark.sql("insert into test_orc_data values(13, '155', 2020)").collect

val df = """select _col2 from test_orc_data limit 5"""
spark.sql(df).collect

```

Its Failing with below error
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, 192.168.0.103, executor driver): java.lang.ArrayIndexOutOfBoundsException: 1
    at org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader.initBatch(OrcColumnarBatchReader.java:156)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat.$anonfun$buildReaderWithPartitionValues$7(OrcFileFormat.scala:258)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:141)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:203)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:116)
    at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.hasNext(DataSourceScanExec.scala:620)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown Source)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
    at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
    at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:343)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:895)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:895)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:372)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:336)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
    at org.apache.spark.scheduler.Task.run(Task.scala:133)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:445)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1489)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:448)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748)`
```

The reason behind this initBatch is not getting the schema that is needed to find out the column value in OrcFileFormat.scala
```
batchReader.initBatch(
 TypeDescription.fromString(resultSchemaString)
```

### Why are the changes needed?
Spark sql queries for orc tables are failing

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

### How was this patch tested?
Unit test is added for this .Also Tested through spark shell and spark submit the failing queries

Closes #29045 from SaurabhChawla100/SPARK-32234.

Lead-authored-by: SaurabhChawla <saurabhc@qubole.com>
Co-authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-16 13:11:47 +00:00
Kent Yao bdeb626c5a [SPARK-32272][SQL] Add SQL standard command SET TIME ZONE
### What changes were proposed in this pull request?

This PR adds the SQL standard command - `SET TIME ZONE` to the current default time zone displacement for the current SQL-session, which is the same as the existing `set spark.sql.session.timeZone=xxx'.

All in all, this PR adds syntax as following,

```
SET TIME ZONE LOCAL;
SET TIME ZONE 'valid time zone';  -- zone offset or region
SET TIME ZONE INTERVAL XXXX; -- xxx must in [-18, + 18] hours, * this range is bigger than ansi  [-14, + 14]
```

### Why are the changes needed?

ANSI compliance and supply pure SQL users a way to retrieve all supported TimeZones

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

yes, add new syntax.

### How was this patch tested?

add unit tests.

and locally verified reference doc

![image](https://user-images.githubusercontent.com/8326978/87510244-c8dc3680-c6a5-11ea-954c-b098be84afee.png)

Closes #29064 from yaooqinn/SPARK-32272.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-16 13:01:53 +00:00
Sean Owen c28a6fa511 [SPARK-29292][SQL][ML] Update rest of default modules (Hive, ML, etc) for Scala 2.13 compilation
### What changes were proposed in this pull request?

Same as https://github.com/apache/spark/pull/29078 and https://github.com/apache/spark/pull/28971 . This makes the rest of the default modules (i.e. those you get without specifying `-Pyarn` etc) compile under Scala 2.13. It does not close the JIRA, as a result. this also of course does not demonstrate that tests pass yet in 2.13.

Note, this does not fix the `repl` module; that's separate.

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

No

### How was this patch tested?

Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12)

Closes #29111 from srowen/SPARK-29292.3.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-15 13:26:28 -07:00
Erik Krogen cf22d947fb [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature
### What changes were proposed in this pull request?

This PR will remove references to these "blacklist" and "whitelist" terms besides the blacklisting feature as a whole, which can be handled in a separate JIRA/PR.

This touches quite a few files, but the changes are straightforward (variable/method/etc. name changes) and most quite self-contained.

### Why are the changes needed?

As per discussion on the Spark dev list, it will be beneficial to remove references to problematic language that can alienate potential community members. One such reference is "blacklist" and "whitelist". While it seems to me that there is some valid debate as to whether these terms have racist origins, the cultural connotations are inescapable in today's world.

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

In the test file `HiveQueryFileTest`, a developer has the ability to specify the system property `spark.hive.whitelist` to specify a list of Hive query files that should be tested. This system property has been renamed to `spark.hive.includelist`. The old property has been kept for compatibility, but will log a warning if used. I am open to feedback from others on whether keeping a deprecated property here is unnecessary given that this is just for developers running tests.

### How was this patch tested?

Existing tests should be suitable since no behavior changes are expected as a result of this PR.

Closes #28874 from xkrogen/xkrogen-SPARK-32036-rename-blacklists.

Authored-by: Erik Krogen <ekrogen@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-15 11:40:55 -05:00
Dongjoon Hyun 8950dcbb1c [SPARK-32318][SQL][TESTS] Add a test case to EliminateSortsSuite for ORDER BY in DISTRIBUTE BY
### What changes were proposed in this pull request?

This PR aims to add a test case to EliminateSortsSuite to protect a valid use case which is using ORDER BY in DISTRIBUTE BY statement.

### Why are the changes needed?

```scala
scala> scala.util.Random.shuffle((1 to 100000).map(x => (x % 2, x))).toDF("a", "b").repartition(2).createOrReplaceTempView("t")

scala> sql("select * from (select * from t order by b) distribute by a").write.orc("/tmp/master")

$ ls -al /tmp/master/
total 56
drwxr-xr-x  10 dongjoon  wheel  320 Jul 14 22:12 ./
drwxrwxrwt  15 root      wheel  480 Jul 14 22:12 ../
-rw-r--r--   1 dongjoon  wheel    8 Jul 14 22:12 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Jul 14 22:12 .part-00000-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel   16 Jul 14 22:12 .part-00043-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel   16 Jul 14 22:12 .part-00191-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Jul 14 22:12 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  119 Jul 14 22:12 part-00000-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc
-rw-r--r--   1 dongjoon  wheel  932 Jul 14 22:12 part-00043-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc
-rw-r--r--   1 dongjoon  wheel  939 Jul 14 22:12 part-00191-2cd3a50e-eded-49a4-b7cf-94e3f090b8c1-c000.snappy.orc
```

The following was found during SPARK-32276. If Spark optimizer removes the inner `ORDER BY`, the file size increases.
```scala
scala> scala.util.Random.shuffle((1 to 100000).map(x => (x % 2, x))).toDF("a", "b").repartition(2).createOrReplaceTempView("t")

scala> sql("select * from (select * from t order by b) distribute by a").write.orc("/tmp/SPARK-32276")

$ ls -al /tmp/SPARK-32276/
total 632
drwxr-xr-x  10 dongjoon  wheel     320 Jul 14 22:08 ./
drwxrwxrwt  14 root      wheel     448 Jul 14 22:08 ../
-rw-r--r--   1 dongjoon  wheel       8 Jul 14 22:08 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel      12 Jul 14 22:08 .part-00000-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel    1188 Jul 14 22:08 .part-00043-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel    1188 Jul 14 22:08 .part-00191-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel       0 Jul 14 22:08 _SUCCESS
-rw-r--r--   1 dongjoon  wheel     119 Jul 14 22:08 part-00000-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc
-rw-r--r--   1 dongjoon  wheel  150735 Jul 14 22:08 part-00043-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc
-rw-r--r--   1 dongjoon  wheel  150741 Jul 14 22:08 part-00191-ba5049f9-b835-49b7-9fdb-bdd11b9891cb-c000.snappy.orc
```

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

No. This only improves the test coverage.

### How was this patch tested?

Pass the GitHub Action or Jenkins.

Closes #29118 from dongjoon-hyun/SPARK-32318.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-15 07:43:56 -07:00
Dilip Biswal e4499932da [SPARK-31480][SQL] Improve the EXPLAIN FORMATTED's output for DSV2's Scan Node
### What changes were proposed in this pull request?
Improve the EXPLAIN FORMATTED output of DSV2 Scan nodes (file based ones).

**Before**
```
== Physical Plan ==
* Project (4)
+- * Filter (3)
   +- * ColumnarToRow (2)
      +- BatchScan (1)

(1) BatchScan
Output [2]: [value#7, id#8]
Arguments: [value#7, id#8], ParquetScan(org.apache.spark.sql.test.TestSparkSession17477bbb,Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml, __spark_hadoop_conf__.xml,org.apache.spark.sql.execution.datasources.InMemoryFileIndexa6c363ce,StructType(StructField(value,IntegerType,true)),StructType(StructField(value,IntegerType,true)),StructType(StructField(id,IntegerType,true)),[Lorg.apache.spark.sql.sources.Filter;40fee459,org.apache.spark.sql.util.CaseInsensitiveStringMapfeca1ec6,Vector(isnotnull(id#8), (id#8 > 1)),List(isnotnull(value#7), (value#7 > 2)))
(2) ...
(3) ...
(4) ...
```
**After**
```
== Physical Plan ==
* Project (4)
+- * Filter (3)
   +- * ColumnarToRow (2)
      +- BatchScan (1)

(1) BatchScan
Output [2]: [value#7, id#8]
DataFilters: [isnotnull(value#7), (value#7 > 2)]
Format: parquet
Location: InMemoryFileIndex[....]
PartitionFilters: [isnotnull(id#8), (id#8 > 1)]
PushedFilers: [IsNotNull(id), IsNotNull(value), GreaterThan(id,1), GreaterThan(value,2)]
ReadSchema: struct<value:int>
(2) ...
(3) ...
(4) ...
```
### Why are the changes needed?
The old format is not very readable. This improves the readability of the plan.

### Does this PR introduce any user-facing change?
Yes. the explain output will be different.

### How was this patch tested?
Added a test case in ExplainSuite.

Closes #28425 from dilipbiswal/dkb_dsv2_explain.

Lead-authored-by: Dilip Biswal <dkbiswal@gmail.com>
Co-authored-by: Dilip Biswal <dkbiswal@apache.org>
Signed-off-by: Dilip Biswal <dkbiswal@apache.org>
2020-07-15 01:28:39 -07:00
Dongjoon Hyun 2527fbc896 Revert "[SPARK-32276][SQL] Remove redundant sorts before repartition nodes"
This reverts commit af8e65fca9.
2020-07-14 22:14:31 -07:00
Jungtaek Lim (HeartSaVioR) 542aefb4c4 [SPARK-31985][SS] Remove incomplete/undocumented stateful aggregation in continuous mode
### What changes were proposed in this pull request?

This removes the undocumented and incomplete feature of "stateful aggregation" in continuous mode, which would reduce 1100+ lines of code.

### Why are the changes needed?

The work for the feature had been stopped for over an year, and no one asked/requested for the availability of such feature in community. Current state for the feature is that it only works with `coalesce(1)` which force the query to read and process, and write in "a" task, which doesn't make sense in production.

The remaining code increases the work on DSv2 changes as well - that's why I don't simply propose reverting relevant commits - the code path has been changed due to DSv2 evolution.

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

Technically no, because it's never documented and can't be used in production in current shape.

### How was this patch tested?

Existing tests.

Closes #29077 from HeartSaVioR/SPARK-31985.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-15 13:40:43 +09:00
Anton Okolnychyi af8e65fca9 [SPARK-32276][SQL] Remove redundant sorts before repartition nodes
### What changes were proposed in this pull request?

This PR removes redundant sorts before repartition nodes with shuffles and repartitionByExpression with deterministic expressions.

### Why are the changes needed?

It looks like our `EliminateSorts` rule can be extended further to remove sorts before repartition nodes that shuffle data as such repartition operations change the ordering and distribution of data. That's why it seems safe to perform the following rewrites:
- `Repartition -> Sort -> Scan` as `Repartition -> Scan`
- `Repartition -> Project -> Sort -> Scan` as `Repartition -> Project -> Scan`

We don't apply this optimization to coalesce as it uses `DefaultPartitionCoalescer` that may preserve the ordering of data if there is no locality info in the parent RDD. At the same time, there is no guarantee that will happen.

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

No.

### How was this patch tested?

More test cases.

Closes #29089 from aokolnychyi/spark-32276.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-14 21:17:33 -07:00
HyukjinKwon 03b5707b51 [MINOR][R] Match collectAsArrowToR with non-streaming collectAsArrowToPython
### What changes were proposed in this pull request?

This PR proposes to port forward #29098 to `collectAsArrowToR`. `collectAsArrowToR` follows `collectAsArrowToPython` in branch-2.4 due to the limitation of ARROW-4512. SparkR vectorization currently cannot use streaming format.

### Why are the changes needed?

For simplicity and consistency.

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

No.

### How was this patch tested?

The same code is being tested in `collectAsArrowToPython` of branch-2.4.

Closes #29100 from HyukjinKwon/minor-parts.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-15 08:46:20 +09:00
yi.wu a47b69a88a [SPARK-32307][SQL] ScalaUDF's canonicalized expression should exclude inputEncoders
### What changes were proposed in this pull request?

Override `canonicalized` to empty the `inputEncoders` for the canonicalized `ScalaUDF`.

### Why are the changes needed?

The following fails on `branch-3.0` currently, not on Apache Spark 3.0.0 release.

```scala
spark.udf.register("key", udf((m: Map[String, String]) => m.keys.head.toInt))
Seq(Map("1" -> "one", "2" -> "two")).toDF("a").createOrReplaceTempView("t")
checkAnswer(sql("SELECT key(a) AS k FROM t GROUP BY key(a)"), Row(1) :: Nil)

[info]   org.apache.spark.sql.AnalysisException: expression 't.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
[info] Aggregate [UDF(a#6)], [UDF(a#6) AS k#8]
[info] +- SubqueryAlias t
[info]    +- Project [value#3 AS a#6]
[info]       +- LocalRelation [value#3]
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:49)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:48)
[info]   at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:130)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:257)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10(CheckAnalysis.scala:259)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10$adapted(CheckAnalysis.scala:259)
[info]   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
[info]   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:259)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10(CheckAnalysis.scala:259)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$10$adapted(CheckAnalysis.scala:259)
[info]   at scala.collection.immutable.List.foreach(List.scala:392)
[info]   at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkValidAggregateExpression$1(CheckAnalysis.scala:259)
...
```

We use the rule`ResolveEncodersInUDF` to resolve `inputEncoders` and the original`ScalaUDF` instance will be updated to a new `ScalaUDF` instance with the resolved encoders at the end. Note, during encoder resolving, types like `map`, `array` will be resolved to new expression(e.g. `MapObjects`, `CatalystToExternalMap`).

However, `ExpressionEncoder` can't be canonicalized. Thus, the canonicalized `ScalaUDF`s become different even if their original  `ScalaUDF`s are the same. Finally, it fails the `checkValidAggregateExpression` when this `ScalaUDF` is used as a group expression.

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

Yes, users will not hit the exception after this fix.

### How was this patch tested?

Added tests.

Closes #29106 from Ngone51/spark-32307.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-14 12:19:01 -07:00
Sean Owen d6a68e0b67 [SPARK-29292][STREAMING][SQL][BUILD] Get streaming, catalyst, sql compiling for Scala 2.13
### What changes were proposed in this pull request?

Continuation of https://github.com/apache/spark/pull/28971 which lets streaming, catalyst and sql compile for 2.13. Same idea.

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

No.

### How was this patch tested?

Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12)

Closes #29078 from srowen/SPARK-29292.2.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-14 02:06:50 -07:00
Liang-Chi Hsieh cc9371d885 [SPARK-32258][SQL] Not duplicate normalization on children for float/double If/CaseWhen/Coalesce
### What changes were proposed in this pull request?

This is followup to #29061. See https://github.com/apache/spark/pull/29061#discussion_r453458611. Basically this moves If/CaseWhen/Coalesce case patterns after float/double case so we don't duplicate normalization on children for float/double If/CaseWhen/Coalesce.

### Why are the changes needed?

Simplify expression tree.

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

No

### How was this patch tested?

Modify unit tests.

Closes #29091 from viirya/SPARK-32258-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-14 05:51:59 +00:00
Peter Toth 24be81689c [SPARK-32241][SQL] Remove empty children of union
### What changes were proposed in this pull request?
This PR removes the empty child relations of a `Union`.

E.g. the query `SELECT c FROM t UNION ALL SELECT c FROM t WHERE false` has the following plan before this PR:
```
== Physical Plan ==
Union
:- *(1) Project [value#219 AS c#222]
:  +- *(1) LocalTableScan [value#219]
+- LocalTableScan <empty>, [c#224]
```
and after this PR:
```
== Physical Plan ==
*(1) Project [value#219 AS c#222]
+- *(1) LocalTableScan [value#219]
```

### Why are the changes needed?
To have a simpler plan.

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

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

Closes #29053 from peter-toth/SPARK-32241-remove-empty-children-of-union.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-14 04:26:29 +00:00
HyukjinKwon 4ad9bfd53b [SPARK-32138] Drop Python 2.7, 3.4 and 3.5
### What changes were proposed in this pull request?

This PR aims to drop Python 2.7, 3.4 and 3.5.

Roughly speaking, it removes all the widely known Python 2 compatibility workarounds such as `sys.version` comparison, `__future__`. Also, it removes the Python 2 dedicated codes such as `ArrayConstructor` in Spark.

### Why are the changes needed?

 1. Unsupport EOL Python versions
 2. Reduce maintenance overhead and remove a bit of legacy codes and hacks for Python 2.
 3. PyPy2 has a critical bug that causes a flaky test, SPARK-28358 given my testing and investigation.
 4. Users can use Python type hints with Pandas UDFs without thinking about Python version
 5. Users can leverage one latest cloudpickle, https://github.com/apache/spark/pull/28950. With Python 3.8+ it can also leverage C pickle.

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

Yes, users cannot use Python 2.7, 3.4 and 3.5 in the upcoming Spark version.

### How was this patch tested?

Manually tested and also tested in Jenkins.

Closes #28957 from HyukjinKwon/SPARK-32138.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-14 11:22:44 +09:00
Holden Karau 90ac9f975b [SPARK-32004][ALL] Drop references to slave
### What changes were proposed in this pull request?

This change replaces the world slave with alternatives matching the context.

### Why are the changes needed?

There is no need to call things slave, we might as well use better clearer names.

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

Yes, the ouput JSON does change. To allow backwards compatibility this is an additive change.
The shell scripts for starting & stopping workers are renamed, and for backwards compatibility old scripts are added to call through to the new ones while printing a deprecation message to stderr.

### How was this patch tested?

Existing tests.

Closes #28864 from holdenk/SPARK-32004-drop-references-to-slave.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-13 14:05:33 -07:00
angerszhu 5521afbd22 [SPARK-32220][SQL][FOLLOW-UP] SHUFFLE_REPLICATE_NL Hint should not change Non-Cartesian Product join result
### What changes were proposed in this pull request?
follow comment https://github.com/apache/spark/pull/29035#discussion_r453468999
Explain for pr

### Why are the changes needed?
add comment

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

### How was this patch tested?
Not need

Closes #29084 from AngersZhuuuu/follow-spark-32220.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-13 08:23:25 -07:00
angerszhu 6d499647b3 [SPARK-32105][SQL] Refactor current ScriptTransformationExec code
### What changes were proposed in this pull request?

 * Renamed  hive transform scrip class `hive/execution/ScriptTransformationExec` to `hive/execution/HiveScriptTransformationExec` (don't rename file)
 * Extract class `BaseScriptTransformationExec ` about common code used across `SparkScriptTransformationExec(next pr add this)` and `HiveScriptTransformationExec`
 * Extract class `BaseScriptTransformationWriterThread` of writing data thread across `SparkScriptTransformationWriterThread(added next for support transform in sql/core )`  and  `HiveScriptTransformationWriterThread` ,
 * `HiveScriptTransformationWriterThread` additionally supports Hive serde format
 * Rename current `Script` strategies in hive module to `HiveScript`, in next pr will add `SparkScript` strategies for support transform in sql/core.

Todo List;

- Support transform in sql/core base on `BaseScriptTransformationExec`, which would run script operator in SQL mode (without Hive).
The output of script would be read as a string and column values are extracted by using a delimiter (default : tab character)
- For Hive, by default only serde's must be used, and without hive we can run without serde
- Cleanup past hacks that are observed (and people suggest / report), such as
       - [Solve string value error about Date/Timestamp in ScriptTransform](https://issues.apache.org/jira/browse/SPARK-31947)
       - [support use transform with aggregation](https://issues.apache.org/jira/browse/SPARK-28227)
       - [support array/map as transform's input](https://issues.apache.org/jira/browse/SPARK-22435)
- Use code-gen projection to serialize rows to output stream()

### Why are the changes needed?
Support run transform in SQL mode without hive

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

### How was this patch tested?
Added UT

Closes #27983 from AngersZhuuuu/follow_spark_15694.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-13 08:58:25 +00:00
Liang-Chi Hsieh b6229df16c [SPARK-32258][SQL] NormalizeFloatingNumbers directly normalizes IF/CaseWhen/Coalesce child expressions
### What changes were proposed in this pull request?

This patch proposes to let `NormalizeFloatingNumbers` rule directly normalizes on certain children expressions. It could simplify expression tree.

### Why are the changes needed?

Currently NormalizeFloatingNumbers rule treats some expressions as black box but we can optimize it a bit by normalizing directly the inner children expressions.

Also see https://github.com/apache/spark/pull/28962#discussion_r448526240.

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

No

### How was this patch tested?

Unit tests.

Closes #29061 from viirya/SPARK-32258.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-12 15:34:43 -07:00
Frank Yin ad90cbff42 [SPARK-31831][SQL][TESTS] Use subclasses for mock in HiveSessionImplSuite
### What changes were proposed in this pull request?
Fix flaky test org.apache.spark.sql.hive.thriftserver.HiveSessionImplSuite by using subclasses to avoid classloader issue.

### Why are the changes needed?
It causes build instability.

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

### How was this patch tested?
It is a fix for a flaky test, but need to run multiple times against Jenkins.

Closes #29069 from frankyin-factual/hive-tests.

Authored-by: Frank Yin <frank@factual.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-13 05:04:47 +09:00
HyukjinKwon c4b0639f83 [SPARK-32270][SQL] Use TextFileFormat in CSV's schema inference with a different encoding
### What changes were proposed in this pull request?

This PR proposes to use text datasource in CSV's schema inference. This shares the same reasons of SPARK-18362, SPARK-19885 and SPARK-19918 - we're currently using Hadoop RDD when the encoding is different, which is unnecessary. This PR completes SPARK-18362, and address the comment at https://github.com/apache/spark/pull/15813#discussion_r90751405.

We should better keep the code paths consistent with existing CSV and JSON datasources as well, but this CSV schema inference with the encoding specified is different from UTF-8 alone.

There can be another story that this PR might lead to a bug fix: Spark session configurations, say Hadoop configurations, are not respected during CSV schema inference when the encoding is different (but it has to be set to Spark context for schema inference when the encoding is different).

### Why are the changes needed?

For consistency, potentially better performance, and fixing a potentially very corner case bug.

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

Virtually no.

### How was this patch tested?

Existing tests should cover.

Closes #29063 from HyukjinKwon/SPARK-32270.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-12 09:44:27 -07:00
Michael Chirico 6ae400ccbe [MINOR][SQL][DOCS] consistency in argument naming for time functions
### What changes were proposed in this pull request?

Rename documented argument `format` as `fmt`, to match the same argument name in several other SQL date/time functions, to wit, `date_format`, `date_trunc`, `trunc`, `to_date`, and `to_timestamp` all use `fmt`. Also `format_string` and `printf` use the same abbreviation in their argument `strfmt`.

### Why are the changes needed?

Consistency -- I was trying to scour the documentation for functions with arguments using Java string formatting, it would have been nice to rely on searching for `fmt` instead of my more manual approach.

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

In the documentation only

### How was this patch tested?

No tests

Closes #29007 from MichaelChirico/sql-doc-format-fmt.

Authored-by: Michael Chirico <michael.chirico@grabtaxi.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-07-12 09:53:27 -05:00
yi.wu 004aea8155 [SPARK-32154][SQL] Use ExpressionEncoder for the return type of ScalaUDF to convert to catalyst type
### What changes were proposed in this pull request?

This PR proposes to use `ExpressionEncoder` for the return type of ScalaUDF to convert to the catalyst type, instead of using `CatalystTypeConverters`.

Note, this change only takes effect for typed Scala UDF since its the only case where we know the type tag of the raw type.

### Why are the changes needed?

Users now could register a UDF with `Instant`/`LocalDate` as return types even with `spark.sql.datetime.java8API.enabled=false`. However, the UDF can not really be used.
For example, if we try:

```scala
scala> sql("set spark.sql.datetime.java8API.enabled=false")
scala> spark.udf.register("buildDate", udf{ d: String => java.time.LocalDate.parse(d) })
scala> Seq("2020-07-02").toDF("d").selectExpr("CAST(buildDate(d) AS STRING)").show
```
Then, we will hit the error:
```scala
java.lang.ClassCastException: java.time.LocalDate cannot be cast to java.sql.Date
  at org.apache.spark.sql.catalyst.CatalystTypeConverters$DateConverter$.toCatalystImpl(CatalystTypeConverters.scala:304)
  at org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:107)
  at org.apache.spark.sql.catalyst.CatalystTypeConverters$.$anonfun$createToCatalystConverter$2(CatalystTypeConverters.scala:425)
  at org.apache.spark.sql.catalyst.expressions.ScalaUDF.eval(ScalaUDF.scala:1169)
...
```
as it actually requires enabling `spark.sql.datetime.java8API.enabled` when using the UDF. And I think this could make users get confused.

This happens because when registering the UDF,  Spark actually uses `ExpressionEncoder` to ser/deser types. However, when using UDF, Spark uses `CatalystTypeConverters`, which is under control of `spark.sql.datetime.java8API.enabled`, to ser/deser types. Therefore, Spark would fail to convert the Java8 date time types.

If we could also use `ExpressionEncoder` to ser/deser types for the return type, similar to what we do for the input parameter types, then, UDF could support Instant/LocalDate, event other combined complex types as well.

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

Yes. Before this PR, if users run the demo above, they would hit the error. After this PR, the demo will run successfully.

### How was this patch tested?

Updated 2 tests and added a new one for combined types of `Instant` and `LocalDate`.

Closes #28979 from Ngone51/udf-return-encoder.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-12 09:45:18 +09:00
Liang-Chi Hsieh 98504e92a7 [SPARK-29358][SQL] Make unionByName optionally fill missing columns with nulls
### What changes were proposed in this pull request?

This patch proposes to make `unionByName` optionally fill missing columns with nulls.

### Why are the changes needed?

Currently, `unionByName` throws exception if detecting different column names between two Datasets. It is strict requirement and sometimes users require more flexible usage that two Datasets with different subset of columns can be union by name resolution.

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

Yes. Adding overloading `Dataset.unionByName` with a boolean parameter that allows different set of column names between two Datasets. Missing columns at each side, will be filled with null values.

### How was this patch tested?

Unit test.

Closes #28996 from viirya/SPARK-29358.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 15:58:40 -07:00