Commit graph

8106 commits

Author SHA1 Message Date
Yuanjian Li f98a063a4b [SPARK-35172][SS] The implementation of RocksDBCheckpointMetadata
### What changes were proposed in this pull request?
Initial implementation of RocksDBCheckpointMetadata. It persists the metadata for RocksDBFileManager.

### Why are the changes needed?
The RocksDBCheckpointMetadata persists the metadata for each committed batch in JSON format. The object contains all RocksDB file names and the number of total keys.
The metadata binds closely with the directory structure of RocksDBFileManager, as described in the design doc - [Directory Structure and Format for Files stored in DFS](https://docs.google.com/document/d/10wVGaUorgPt4iVe4phunAcjU924fa3-_Kf29-2nxH6Y/edit#heading=h.zgvw85ijoz2).

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

### How was this patch tested?
New UT added.

Closes #32272 from xuanyuanking/SPARK-35172.

Lead-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Co-authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-27 22:56:50 +09:00
dgd-contributor 52a1f8c000 [SPARK-33428][SQL] Match the behavior of conv function to MySQL's
### What changes were proposed in this pull request?
Spark conv function is from MySQL and it's better to follow the MySQL behavior. MySQL returns the max unsigned long if the input string is too big, and Spark should follow it.

However, seems Spark has different behavior in two cases:

MySQL allows leading spaces but Spark does not.
If the input string is way too long, Spark fails with ArrayIndexOutOfBoundException

This patch now help conv follow behavior in those two cases
conv allows leading spaces
conv will return the max unsigned long when the input string is way too long

### Why are the changes needed?
fixing it to match the behavior of conv function to the (almost) only one reference of another DBMS, MySQL

### Does this PR introduce _any_ user-facing change?
Yes, as pointed out above

### How was this patch tested?
Add test

Closes #32684 from dgd-contributor/SPARK-33428.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-27 12:12:39 +00:00
Gengliang Wang 5bcd1c29f0 [SPARK-35535][SQL] New data source V2 API: LocalScan
### What changes were proposed in this pull request?

Add a new data source V2 API: `LocalScan`. It is a special Scan that will happen on Driver locally instead of Executors.

### Why are the changes needed?

The new API improves the flexibility of the DSV2 API. It allows developers to implement connectors for data sources of small data sizes.
For example, we can build a data source for Spark History applications from Spark History Server RESTFUL API. The result set is small and fetching all the results from the Spark driver is good enough. Making it a data source allows us to operate SQL queries with filters or table joins.

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

No.

### How was this patch tested?

Unit test

Closes #32678 from gengliangwang/LocalScan.

Lead-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-27 19:31:56 +09:00
Cheng Su 5cc17ba0c7 [SPARK-35351][SQL][FOLLOWUP] Avoid using loaded variable for LEFT ANTI SMJ code-gen
### What changes were proposed in this pull request?

This is a followup from https://github.com/apache/spark/pull/32547#discussion_r639916474, where for LEFT ANTI join, we do not need to depend on `loaded` variable, as in `codegenAnti` we only load `streamedAfter` no more than once (i.e. assign column values from streamed row which are not used in join condition).

### Why are the changes needed?

Avoid unnecessary processing in code-gen (though it's just `boolean $loaded = false;`, and `if (!$loaded) { $loaded = true; }`).

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

No.

### How was this patch tested?

Existing unite tests in `ExistenceJoinSuite`.

Closes #32681 from c21/join-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-27 04:59:54 +00:00
ulysses-you dc7b5a99f0 [SPARK-35282][SQL] Support AQE side shuffled hash join formula using rule
### What changes were proposed in this pull request?

The main code change is:
* Change rule `DemoteBroadcastHashJoin` to `DynamicJoinSelection` and add shuffle hash join selection code.
* Specify a join strategy hint `SHUFFLE_HASH` if AQE think a join can be converted to SHJ.
* Skip `preferSortMerge` config check in AQE side if a join can be converted to SHJ.

### Why are the changes needed?

Use AQE runtime statistics to decide if we can use shuffled hash join instead of sort merge join. Currently, the formula of shuffled hash join selection dose not work due to the dymanic shuffle partition number.

Add a new config spark.sql.adaptive.shuffledHashJoinLocalMapThreshold to decide if join can be converted to shuffled hash join safely.

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

Yes, add a new config.

### How was this patch tested?

Add test.

Closes #32550 from ulysses-you/SPARK-35282-2.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 14:16:04 +00:00
Cheng Su dd677770d8 [SPARK-35529][SQL] Add fallback metrics for hash aggregate
### What changes were proposed in this pull request?

Add the metrics to record how many tasks fallback to sort-based aggregation for hash aggregation. This will help developers and users to debug and optimize query. Object hash aggregation has similar metrics already.

### Why are the changes needed?

Help developers and users to debug and optimize query with hash aggregation.

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

Yes, the added metrics will show up in Spark web UI.
Example:
<img width="604" alt="Screen Shot 2021-05-26 at 12 17 08 AM" src="https://user-images.githubusercontent.com/4629931/119618437-bf3c5880-bdb7-11eb-89bb-5b88db78639f.png">

### How was this patch tested?

Changed unit test in `SQLMetricsSuite.scala`.

Closes #32671 from c21/agg-metrics.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 11:28:12 +00:00
itholic 79a6b0cc8a [SPARK-35509][DOCS] Move text data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move text data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for text data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "Text Files" page
<img width="823" alt="Screen Shot 2021-05-26 at 3 20 11 PM" src="https://user-images.githubusercontent.com/44108233/119611669-f5202200-be35-11eb-9307-45846949d300.png">

- Python
<img width="791" alt="Screen Shot 2021-05-25 at 5 04 26 PM" src="https://user-images.githubusercontent.com/44108233/119462469-b9c11d00-bd7b-11eb-8f19-2ba7b9ceb318.png">

- Scala
<img width="683" alt="Screen Shot 2021-05-25 at 5 05 10 PM" src="https://user-images.githubusercontent.com/44108233/119462483-bd54a400-bd7b-11eb-8177-74e4d7035e63.png">

- Java
<img width="665" alt="Screen Shot 2021-05-25 at 5 05 36 PM" src="https://user-images.githubusercontent.com/44108233/119462501-bfb6fe00-bd7b-11eb-8161-12c58fabe7e2.png">

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32660 from itholic/SPARK-35509.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-26 17:12:49 +09:00
Vinod KC e3c6907c99 [SPARK-35490][BUILD] Update json4s to 3.7.0-M11
### What changes were proposed in this pull request?
This PR aims to upgrade json4s from   3.7.0-M5  to 3.7.0-M11

Note: json4s version greater than 3.7.0-M11 is not binary compatible with Spark third party jars

### Why are the changes needed?
Multiple defect fixes and improvements  like

https://github.com/json4s/json4s/issues/750
https://github.com/json4s/json4s/issues/554
https://github.com/json4s/json4s/issues/715

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

### How was this patch tested?
Ran with the existing UTs

Closes #32636 from vinodkc/br_build_upgrade_json4s.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-26 11:10:14 +03:00
Linhong Liu af1dba7ca5 [SPARK-35440][SQL] Add function type to ExpressionInfo for UDF
### What changes were proposed in this pull request?
Add the function type, such as "scala_udf", "python_udf", "java_udf", "hive", "built-in" to the `ExpressionInfo` for UDF.

### Why are the changes needed?
Make the `ExpressionInfo` of UDF more meaningful

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

### How was this patch tested?
existing and newly added UT

Closes #32587 from linhongliu-db/udf-language.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 04:40:53 +00:00
Wenchen Fan 859a53424a [SPARK-35447][SQL] Optimize skew join before coalescing shuffle partitions
### What changes were proposed in this pull request?

This PR improves the interaction between partition coalescing and skew handling by moving the skew join rule ahead of the partition coalescing rule and making corresponding changes to the two rules:
1. Simplify `OptimizeSkewedJoin` as it doesn't need to handle `CustomShuffleReaderExec` anymore.
2. Update `CoalesceShufflePartitions` to support coalescing non-skewed partitions.

### Why are the changes needed?

It's a bit hard to reason about skew join if partitions have been coalesced. A skewed partition needs to be much larger than other partitions and we need to look at the raw sizes before coalescing.

It also makes `OptimizeSkewedJoin` more robust, as we don't need to worry about a skewed partition being coalesced with a small partition and breaks skew join handling.

It also helps with https://github.com/apache/spark/pull/31653 , which needs to move `OptimizeSkewedJoin` to an earlier phase and run before `CoalesceShufflePartitions`.

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

No

### How was this patch tested?

new UT and existing tests

Closes #32594 from cloud-fan/shuffle.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 13:12:45 +00:00
ulysses-you 631077db08 [SPARK-35455][SQL] Unify empty relation optimization between normal and AQE optimizer
### What changes were proposed in this pull request?

* remove `EliminateUnnecessaryJoin`, using `AQEPropagateEmptyRelation` instead.
* eliminate join, aggregate, limit, repartition, sort, generate which is beneficial.

### Why are the changes needed?

Make `EliminateUnnecessaryJoin` available with more case.

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

No.

### How was this patch tested?

Add test.

Closes #32602 from ulysses-you/SPARK-35455.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 08:59:59 +00:00
Kousuke Saruta d4fb98354a [SPARK-35287][SQL] Allow RemoveRedundantProjects to preserve ProjectExec which generates UnsafeRow for DataSourceV2ScanRelation
### What changes were proposed in this pull request?

This PR fixes an issue that `RemoveRedundantProjects` removes `ProjectExec` which is for generating `UnsafeRow`.
In `DataSourceV2Strategy`, `ProjectExec` will be inserted to ensure internal rows are `UnsafeRow`.

```
  private def withProjectAndFilter(
      project: Seq[NamedExpression],
      filters: Seq[Expression],
      scan: LeafExecNode,
      needsUnsafeConversion: Boolean): SparkPlan = {
    val filterCondition = filters.reduceLeftOption(And)
    val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan)

    if (withFilter.output != project || needsUnsafeConversion) {
      ProjectExec(project, withFilter)
    } else {
      withFilter
    }
  }
...
    case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) =>
      // projection and filters were already pushed down in the optimizer.
      // this uses PhysicalOperation to get the projection and ensure that if the batch scan does
      // not support columnar, a projection is added to convert the rows to UnsafeRow.
      val batchExec = BatchScanExec(relation.output, relation.scan)
      withProjectAndFilter(project, filters, batchExec, !batchExec.supportsColumnar) :: Nil
```
So, the hierarchy of the partial tree should be like `ProjectExec(FilterExec(BatchScan))`.
But `RemoveRedundantProjects` doesn't consider this type of hierarchy, leading `ClassCastException`.

A concreate example to reproduce this issue is reported:
```
import scala.collection.JavaConverters._

import org.apache.iceberg.{PartitionSpec, TableProperties}
import org.apache.iceberg.hadoop.HadoopTables
import org.apache.iceberg.spark.SparkSchemaUtil
import org.apache.spark.sql.{DataFrame, QueryTest, SparkSession}
import org.apache.spark.sql.internal.SQLConf

class RemoveRedundantProjectsTest extends QueryTest {
  override val spark: SparkSession = SparkSession
    .builder()
    .master("local[4]")
    .config("spark.driver.bindAddress", "127.0.0.1")
    .appName(suiteName)
    .getOrCreate()
  test("RemoveRedundantProjects removes non-redundant projects") {
    withSQLConf(
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
      SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false",
      SQLConf.REMOVE_REDUNDANT_PROJECTS_ENABLED.key -> "true") {
      withTempDir { dir =>
        val path = dir.getCanonicalPath
        val data = spark.range(3).toDF
        val table = new HadoopTables().create(
          SparkSchemaUtil.convert(data.schema),
          PartitionSpec.unpartitioned(),
          Map(TableProperties.WRITE_NEW_DATA_LOCATION -> path).asJava,
          path)
        data.write.format("iceberg").mode("overwrite").save(path)
        table.refresh()

        val df = spark.read.format("iceberg").load(path)
        val dfX = df.as("x")
        val dfY = df.as("y")
        val join = dfX.filter(dfX("id") > 0).join(dfY, "id")
        join.explain("extended")
        assert(join.count() == 2)
      }
    }
  }
}
```
```
[info] - RemoveRedundantProjects removes non-redundant projects *** FAILED ***
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 4) (xeroxms100.northamerica.corp.microsoft.com executor driver): java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow
[info]  at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:226)
[info]  at org.apache.spark.sql.execution.SortExec.$anonfun$doExecute$1(SortExec.scala:119)
```

### Why are the changes needed?

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

No.

### How was this patch tested?

New test.

Closes #32606 from sarutak/fix-project-removal-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 00:26:10 +08:00
Chao Sun c709efc1e7 [SPARK-34981][SQL][FOLLOWUP] Use SpecificInternalRow in ApplyFunctionExpression
### What changes were proposed in this pull request?

Use `SpecificInternalRow` instead of `GenericInternalRow` to avoid boxing / unboxing cost.

### Why are the changes needed?

Since it doesn't know the input row schema, `GenericInternalRow` potentially need to apply boxing for input arguments. It's better to use `SpecificInternalRow` instead since we know input data types.

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

No.

### How was this patch tested?

Existing tests.

Closes #32647 from sunchao/specific-input-row.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-24 17:25:24 +09:00
Adam Binford 6c0c617bd0 [SPARK-35449][SQL] Only extract common expressions from CaseWhen values if elseValue is set
### What changes were proposed in this pull request?

This PR fixes a bug with subexpression elimination for CaseWhen statements. https://github.com/apache/spark/pull/30245 added support for creating subexpressions that are present in all branches of conditional statements. However, for a statement to be in "all branches" of a CaseWhen statement, it must also be in the elseValue.

### Why are the changes needed?

Fix a bug where a subexpression can be created and run for branches of a conditional that don't pass. This can cause issues especially with a UDF in a branch that gets executed assuming the condition is true.

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

Yes, fixes a potential bug where a UDF could be eagerly executed even though it might expect to have already passed some form of validation. For example:
```
val col = when($"id" < 0, myUdf($"id"))
spark.range(1).select(when(col > 0, col)).show()
```

`myUdf($"id")` is considered a subexpression and eagerly evaluated, because it is pulled out as a common expression from both executions of the when clause, but if `id >= 0` it should never actually be run.

### How was this patch tested?

Updated existing test with new case.

Closes #32595 from Kimahriman/bug-case-subexpr-elimination.

Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-24 00:27:41 -07:00
Liang-Chi Hsieh 9e1b204bcc [SPARK-35410][SQL] SubExpr elimination should not include redundant children exprs in conditional expression
### What changes were proposed in this pull request?

This patch fixes a bug when dealing with common expressions in conditional expressions such as `CaseWhen` during subexpression elimination.

For example, previously we find common expressions among conditions of `CaseWhen`, but children expressions are also counted into. We should not count these children expressions as common expressions.

### Why are the changes needed?

If the redundant children expressions are counted as common expressions too, they will be redundantly evaluated and miss the subexpression elimination opportunity.

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

No

### How was this patch tested?

Added tests.

Closes #32559 from viirya/SPARK-35410.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-23 08:24:44 -07:00
Hyukjin Kwon 1d9f09decb [SPARK-35480][SQL] Make percentile_approx work with pivot
### What changes were proposed in this pull request?

This PR proposes to avoid wrapping if-else to the constant literals for `percentage` and `accuracy` in `percentile_approx`. They are expected to be literals (or foldable expressions).

Pivot works by two phrase aggregations, and it works with manipulating the input to `null` for non-matched values (pivot column and value).

Note that pivot supports an optimized version without such logic with changing input to `null` for some types (non-nested types basically). So the issue fixed by this PR is only for complex types.

```scala
val df = Seq(
  ("a", -1.0), ("a", 5.5), ("a", 2.5), ("b", 3.0), ("b", 5.2)).toDF("type", "value")
  .groupBy().pivot("type", Seq("a", "b")).agg(
    percentile_approx(col("value"), array(lit(0.5)), lit(10000)))
df.show()
```

**Before:**

```
org.apache.spark.sql.AnalysisException: cannot resolve 'percentile_approx((IF((type <=> CAST('a' AS STRING)), value, CAST(NULL AS DOUBLE))), (IF((type <=> CAST('a' AS STRING)), array(0.5D), NULL)), (IF((type <=> CAST('a' AS STRING)), 10000, CAST(NULL AS INT))))' due to data type mismatch: The accuracy or percentage provided must be a constant literal;
'Aggregate [percentile_approx(if ((type#7 <=> cast(a as string))) value#8 else cast(null as double), if ((type#7 <=> cast(a as string))) array(0.5) else cast(null as array<double>), if ((type#7 <=> cast(a as string))) 10000 else cast(null as int), 0, 0) AS a#16, percentile_approx(if ((type#7 <=> cast(b as string))) value#8 else cast(null as double), if ((type#7 <=> cast(b as string))) array(0.5) else cast(null as array<double>), if ((type#7 <=> cast(b as string))) 10000 else cast(null as int), 0, 0) AS b#18]
+- Project [_1#2 AS type#7, _2#3 AS value#8]
   +- LocalRelation [_1#2, _2#3]
```

**After:**

```
+-----+-----+
|    a|    b|
+-----+-----+
|[2.5]|[3.0]|
+-----+-----+
```

### Why are the changes needed?

To make percentile_approx work with pivot as expected

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

Yes. It threw an exception but now it returns a correct result as shown above.

### How was this patch tested?

Manually tested and unit test was added.

Closes #32619 from HyukjinKwon/SPARK-35480.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-23 07:35:43 +09:00
Wenchen Fan b624b7e93f [SPARK-28551][SQL][FOLLOWUP] Use the corrected hadoop conf
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32411, to fix a mistake and use `sparkSession.sessionState.newHadoopConf` which includes SQL configs instead of `sparkSession.sparkContext.hadoopConfiguration` .

### Why are the changes needed?

fix mistake

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

no

### How was this patch tested?

existing tests

Closes #32618 from cloud-fan/follow1.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-22 10:33:57 +08:00
Dongjoon Hyun cc05daa884 [SPARK-34558][SQL][TESTS][FOLLOWUP] Fix a test to use a unknown filesystem
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32622 to fix a test case.

### Why are the changes needed?

Fix a wrong test case name and fix the test case to cause the expected error correctly.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #32623 from dongjoon-hyun/SPARK-34558.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-21 10:13:49 -07:00
Wenchen Fan 7274e3a4d2 [SPARK-34558][SQL][FOLLOWUP] Do not fail Spark startup with a broken FileSystem
### What changes were proposed in this pull request?

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

https://github.com/apache/spark/pull/31671 qualifies the warehouse at the beginning, which may fail Spark startup if something goes wrong, like the underlying FileSystem can't be initialized.

This PR falls back to the old behavior and leave the warehouse path unqualified if qualifying fails.

### Why are the changes needed?

Fix a regression. It's important to be always able to start Spark app (e.g. spark-shell), so that we can debug.

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

No

### How was this patch tested?

a new test case

Closes #32622 from cloud-fan/follow2.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-21 08:51:10 -07:00
itholic d2bdd6595e [SPARK-35025][SQL][PYTHON][DOCS] Move Parquet data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move Parquet data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for Parquet data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "Parquet Files" page
![Screen Shot 2021-05-21 at 1 35 08 PM](https://user-images.githubusercontent.com/44108233/119082866-e7375f00-ba39-11eb-9ade-a931a5957b34.png)

- Python
![Screen Shot 2021-05-21 at 1 38 27 PM](https://user-images.githubusercontent.com/44108233/119082879-eef70380-ba39-11eb-9e8e-ee50eed98dbe.png)

- Scala
![Screen Shot 2021-05-21 at 1 36 52 PM](https://user-images.githubusercontent.com/44108233/119082884-f1595d80-ba39-11eb-98d5-966657df65f7.png)

- Java
![Screen Shot 2021-05-21 at 1 37 19 PM](https://user-images.githubusercontent.com/44108233/119082888-f4544e00-ba39-11eb-8bf8-47ce78ec0b01.png)

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32161 from itholic/SPARK-34491.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 18:05:49 +09:00
itholic 419ddcb2a4 [SPARK-34494][SQL][DOCS] Move JSON data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move JSON data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for JSON data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "JSON Files" page
<img width="876" alt="Screen Shot 2021-05-20 at 8 48 27 PM" src="https://user-images.githubusercontent.com/44108233/118973662-ddb3e580-b9ac-11eb-987c-8139aa9c3fe2.png">

- Python
<img width="714" alt="Screen Shot 2021-04-16 at 5 04 11 PM" src="https://user-images.githubusercontent.com/44108233/114992491-ca0cef00-9ed5-11eb-9d0f-4de60d8b2516.png">

- Scala
<img width="726" alt="Screen Shot 2021-04-16 at 5 04 54 PM" src="https://user-images.githubusercontent.com/44108233/114992594-e315a000-9ed5-11eb-8bd3-af7e568fcfe1.png">

- Java
<img width="911" alt="Screen Shot 2021-04-16 at 5 06 11 PM" src="https://user-images.githubusercontent.com/44108233/114992751-10624e00-9ed6-11eb-888c-8668d3c74289.png">

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32204 from itholic/SPARK-35081.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 18:05:13 +09:00
itholic 0fe65b5365 [SPARK-35395][DOCS] Move ORC data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move ORC data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for ORC data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "ORC Files" page
![Screen Shot 2021-05-21 at 2 07 14 PM](https://user-images.githubusercontent.com/44108233/119085078-f4564d00-ba3d-11eb-8990-3ba031d809da.png)

- Python
![Screen Shot 2021-05-21 at 2 06 46 PM](https://user-images.githubusercontent.com/44108233/119085097-00daa580-ba3e-11eb-8017-ac5a95a7c053.png)

- Scala
![Screen Shot 2021-05-21 at 2 06 09 PM](https://user-images.githubusercontent.com/44108233/119085135-164fcf80-ba3e-11eb-9cac-78dded523f38.png)

- Java
![Screen Shot 2021-05-21 at 2 06 30 PM](https://user-images.githubusercontent.com/44108233/119085125-118b1b80-ba3e-11eb-9434-f26612d7da13.png)

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32546 from itholic/SPARK-35395.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 18:03:57 +09:00
yi.wu 34284c0649 [SPARK-35454][SQL] One LogicalPlan can match multiple dataset ids
### What changes were proposed in this pull request?

Change the type of `DATASET_ID_TAG` from `Long` to `HashSet[Long]` to allow the logical plan to match multiple datasets.

### Why are the changes needed?

During the transformation from one Dataset to another Dataset, the DATASET_ID_TAG of logical plan won't change if the plan itself doesn't change:

b5241c97b1/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala (L234-L237)

However, dataset id always changes even if the logical plan doesn't change:
b5241c97b1/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala (L207-L208)

And this can lead to the mismatch between dataset's id and col's __dataset_id. E.g.,

```scala
  test("SPARK-28344: fail ambiguous self join - Dataset.colRegex as column ref") {
    // The test can fail if we change it to:
    // val df1 = spark.range(3).toDF()
    // val df2 = df1.filter($"id" > 0).toDF()
    val df1 = spark.range(3)
    val df2 = df1.filter($"id" > 0)

    withSQLConf(
      SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true",
      SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
      assertAmbiguousSelfJoin(df1.join(df2, df1.colRegex("id") > df2.colRegex("id")))
    }
  }
```

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

No.

### How was this patch tested?

Added unit tests.

Closes #32616 from Ngone51/fix-ambiguous-join.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 16:15:17 +08:00
Max Gekk 2b08070e79 [SPARK-35427][SQL][TESTS] Check the EXCEPTION rebase mode for Avro/Parquet
### What changes were proposed in this pull request?
Add tests to check the `EXCEPTION` rebase mode explicitly in the datasources:
- Parquet: `DATE` type and `TIMESTAMP`: `INT96`, `TIMESTAMP_MICROS`, `TIMESTAMP_MILLIS`
- Avro: `DATE` type and `TIMESTAMP`: `timestamp-millis` and `timestamp-micros`.

### Why are the changes needed?
1. To improve test coverage
2. The `EXCEPTION` rebase mode should be checked independently from the default settings.

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

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "test:testOnly *AvroV2Suite"
$ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite"
```

Closes #32574 from MaxGekk/test-rebase-exception.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 06:18:06 +00:00
gengjiaan c740c097e0 [SPARK-35063][SQL] Group exception messages in sql/catalyst
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32478 from beliefer/SPARK-35063.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 06:15:26 +00:00
Takeshi Yamamuro 1a923f5319 [SPARK-35479][SQL] Format PartitionFilters IN strings in scan nodes
### What changes were proposed in this pull request?

This PR proposes to format strings correctly for `PushedFilters`. For example, `explain()` for a query below prints `v in (array('a'))` as `PushedFilters: [In(v, [WrappedArray(a)])]`;

```
scala> sql("create table t (v array<string>) using parquet")
scala> sql("select * from t where v in (array('a'), null)").explain()
== Physical Plan ==
*(1) Filter v#4 IN ([a],null)
+- FileScan parquet default.t[v#4] Batched: false, DataFilters: [v#4 IN ([a],null)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-3.1.1-bin-hadoop2.7/spark-warehouse/t], PartitionFilters: [], PushedFilters: [In(v, [WrappedArray(a),null])], ReadSchema: struct<v:array<string>>
```
This PR makes `explain()` print it as `PushedFilters: [In(v, [[a]])]`;
```
scala> sql("select * from t where v in (array('a'), null)").explain()
== Physical Plan ==
*(1) Filter v#4 IN ([a],null)
+- FileScan parquet default.t[v#4] Batched: false, DataFilters: [v#4 IN ([a],null)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-3.1.1-bin-hadoop2.7/spark-warehouse/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct<v:array<string>>
```
NOTE: This PR includes a bugfix caused by #32577 (See the cloud-fan comment: https://github.com/apache/spark/pull/32577/files#r636108150).

### Why are the changes needed?

To improve explain strings.

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

Yes, this PR improves the explain strings for pushed-down filters.

### How was this patch tested?

Added tests in `SQLQueryTestSuite`.

Closes #32615 from maropu/ExplainPartitionFilters.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 05:45:45 +00:00
Kent Yao 2e9936db93 [SPARK-35456][CORE] Print the invalid value in config validation error message
### What changes were proposed in this pull request?

Print the invalid value in config validation error message for `checkValue` just like `checkValues`

### Why are the changes needed?

Invalid configuration values may come in many ways, this PR can help different kinds of users or developers to identify what the config the error is related to

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

yes, but only error msg
### How was this patch tested?

yes, modified tests

Closes #32600 from yaooqinn/SPARK-35456.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 14:22:29 +09:00
Vinod KC bdd8e1dbb1 [SPARK-28551][SQL] CTAS with LOCATION should not allow to a non-empty directory
### What changes were proposed in this pull request?

CTAS with location clause acts as an insert overwrite. This can cause problems when there are subdirectories within a location directory.
This causes some users to accidentally wipe out directories with very important data. We should not allow CTAS with location to a non-empty directory.

### Why are the changes needed?

Hive already handled this scenario: HIVE-11319

Steps to reproduce:

```scala
sql("""create external table  `demo_CTAS`( `comment` string) PARTITIONED BY (`col1` string, `col2` string) STORED AS parquet location '/tmp/u1/demo_CTAS'""")
sql("""INSERT OVERWRITE TABLE demo_CTAS partition (col1='1',col2='1') VALUES ('abc')""")
sql("select* from demo_CTAS").show
sql("""create table ctas1 location '/tmp/u2/ctas1' as select * from demo_CTAS""")
sql("select* from ctas1").show
sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""")
```

Before the fix: Both create table operations will succeed. But values in table ctas1 will be replaced by ctas2 accidentally.

After the fix: `create table ctas2...` will throw `AnalysisException`:

```
org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true.
```

### Does this PR introduce _any_ user-facing change?
Yes, if the location directory is not empty, CTAS with location will throw AnalysisException

```
sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""")
```
```
org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true.
```

`CREATE TABLE AS SELECT` with non-empty `LOCATION` will throw `AnalysisException`. To restore the behavior before Spark 3.2, need to  set `spark.sql.legacy.allowNonEmptyLocationInCTAS` to `true`. , default value is `false`.
Updated SQL migration guide.

### How was this patch tested?
Test case added in SQLQuerySuite.scala

Closes #32411 from vinodkc/br_fixCTAS_nonempty_dir.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-20 06:13:18 +00:00
Cheng Su 586caae3cc [SPARK-35438][SQL][DOCS] Minor documentation fix for window physical operator
### What changes were proposed in this pull request?

As title. Fixed two places where the documentation for window operator has some error.

### Why are the changes needed?

Help people read code for window operator more easily in the future.

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

No.

### How was this patch tested?

Existing tests.

Closes #32585 from c21/minor-doc.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-20 08:47:19 +09:00
Andy Grove 52e3cf9ff5 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use
### What changes were proposed in this pull request?
AQE has an optimization where it attempts to reuse compatible exchanges but it does not take into account whether the exchanges are columnar or not, resulting in incorrect reuse under some circumstances.

This PR simply changes the key used to lookup cached stages. It now uses the canonicalized form of the new query stage (potentially created by a plugin) rather than using the canonicalized form of the original exchange.

### Why are the changes needed?
When using the [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) we sometimes see a new query stage correctly create a row-based exchange and then Spark replaces it with a cached columnar exchange, which is not compatible, and this causes queries to fail.

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

### How was this patch tested?
The patch has been tested with the query that highlighted this issue. I looked at writing unit tests for this but it would involve implementing a mock columnar exchange in the tests so would be quite a bit of work. If anyone has ideas on other ways to test this I am happy to hear them.

Closes #32195 from andygrove/SPARK-35093.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-05-19 07:45:26 -05:00
Kousuke Saruta 9283bebbbd [SPARK-35418][SQL] Add sentences function to functions.{scala,py}
### What changes were proposed in this pull request?

This PR adds `sentences`, a string function, which is present as of `2.0.0` but missing in `functions.{scala,py}`.

### Why are the changes needed?

This function can be only used from SQL for now.
It's good if we can use this function from Scala/Python code as well as SQL.

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

Yes. Users can use this function from Scala and Python.

### How was this patch tested?

New test.

Closes #32566 from sarutak/sentences-function.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-19 20:07:28 +09:00
Yuzhou Sun a72d05c7e6 [SPARK-35106][CORE][SQL] Avoid failing rename caused by destination directory not exist
### What changes were proposed in this pull request?

1. In HadoopMapReduceCommitProtocol, create parent directory before renaming custom partition path staging files
2. In InMemoryCatalog and HiveExternalCatalog, create new partition directory before renaming old partition path
3. Check return value of FileSystem#rename, if false, throw exception to avoid silent data loss cause by rename failure
4. Change DebugFilesystem#rename behavior to make it match HDFS's behavior (return false without rename when dst parent directory not exist)

### Why are the changes needed?

Depends on FileSystem#rename implementation, when destination directory does not exist, file system may
1. return false without renaming file nor throwing exception (e.g. HDFS), or
2. create destination directory, rename files, and return true (e.g. LocalFileSystem)

In the first case above, renames in HadoopMapReduceCommitProtocol for custom partition path will fail silently if the destination partition path does not exist. Failed renames can happen when
1. dynamicPartitionOverwrite == true, the custom partition path directories are deleted by the job before the rename; or
2. the custom partition path directories do not exist before the job; or
3. something else is wrong when file system handle `rename`

The renames in MemoryCatalog and HiveExternalCatalog for partition renaming also have similar issue.

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

No

### How was this patch tested?

Modified DebugFilesystem#rename, and added new unit tests.

Without the fix in src code, five InsertSuite tests and one AlterTableRenamePartitionSuite test failed:
InsertSuite.SPARK-20236: dynamic partition overwrite with custom partition path (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: insert overwrite with custom partition path
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: dynamic partition overwrite with custom partition path
```
== Results ==
!== Correct Answer - 2 ==   == Spark Answer - 1 ==
!struct<>                   struct<i:int,part1:int,part2:int>
 [1,1,1]                    [1,1,1]
![1,1,2]
```

InsertSuite.SPARK-35106: Throw exception when rename custom partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

InsertSuite.SPARK-35106: Throw exception when rename dynamic partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

AlterTableRenamePartitionSuite.ALTER TABLE .. RENAME PARTITION V1: multi part partition (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
 struct<>                   struct<>
![3,123,3]
```

Closes #32530 from YuzhouSun/SPARK-35106.

Authored-by: Yuzhou Sun <yuzhosun@amazon.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 15:46:27 +08:00
Kousuke Saruta 0b3758e8cd [SPARK-35421][SS] Remove redundant ProjectExec from streaming queries with V2Relation
### What changes were proposed in this pull request?

This PR fixes an issue that streaming queries with V2Relation can have redundant `ProjectExec` in its physical plan.
You can easily reproduce this issue with the following code.
```
import org.apache.spark.sql.streaming.Trigger

val query = spark.
  readStream.
  format("rate").
  option("rowsPerSecond", 1000).
  option("rampUpTime", "10s").
  load().
  selectExpr("timestamp", "100",  "value").
  writeStream.
  format("console").
  trigger(Trigger.ProcessingTime("5 seconds")).
  // trigger(Trigger.Continuous("5 seconds")). // You can reproduce with continuous processing too.
  outputMode("append").
  start()
```
The plan tree is here.
![ss-before](https://user-images.githubusercontent.com/4736016/118454996-ec439800-b733-11eb-8cd8-ed8af73a91b8.png)

### Why are the changes needed?

For better performance.

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

No.

### How was this patch tested?

I run the same code above and get the following plan tree.
![ss-after](https://user-images.githubusercontent.com/4736016/118455755-1bf2a000-b734-11eb-999e-4b8c19ad34d7.png)

Closes #32570 from sarutak/fix-redundant-projectexec.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 06:19:01 +00:00
Takeshi Yamamuro 746d80d87a [SPARK-35422][SQL] Fix plan-printing issues to pass the TPCDS plan stability tests in Scala v2.13
### What changes were proposed in this pull request?

To pass the TPCDS-related plan stability tests in scala-2.13, this PR proposes to fix two things below;
 - (1) Sorts elements in the predicate `InSet` and the source filter `In` for printing their nodes.
 - (2) Formats nested collection elements (`Seq`, `Array`, and `Set`) recursively in `TreeNode.argString`.

As for (1), it seems v2.12/v2.13 prints `Set` elements with a different order, so we need to sort them explicitly. As for (2), the `Seq` implementation is different between v2.12/v2.13, so we need to format nested  `Seq` elements correctly to hide the name of its implementation (See an example below);
```
 (74) Expand [codegen id : 20]
 Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45]
-Arguments: [ArrayBuffer(sales#41, returns#42, ...    <-- scala-2.12
+Arguments: [Vector(sales#41, returns#42, ...         <-- scala-2.13

+Arguments: [[(sales#41, returns#42, ...              <--  the proposed fix to hide the name of its implementation
```

### Why are the changes needed?

To pass the tests in Scala v2.13.

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

Yes, this fix changes query explain strings.

### How was this patch tested?

Manually checked.

Closes #32577 from maropu/FixTPCDSTestIssueInScala213.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 08:50:00 +00:00
Chao Sun 44d762abc6 [SPARK-35389][SQL] V2 ScalarFunction should support magic method with null arguments
### What changes were proposed in this pull request?

When creating `Invoke` and `StaticInvoke` for `ScalarFunction`'s magic method, set `propagateNull` to false.

### Why are the changes needed?

When `propgagateNull` is true (which is the default value), `Invoke` and `StaticInvoke` will return null if any of the argument is null. For scalar function this is incorrect, as we should leave the logic to function implementation instead.

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

Yes. Now null arguments shall be properly handled with magic method.

### How was this patch tested?

Added new tests.

Closes #32553 from sunchao/SPARK-35389.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 08:45:55 +00:00
Cheng Su cce0048c78 [SPARK-35351][SQL] Add code-gen for left anti sort merge join
### What changes were proposed in this pull request?

As title. This PR is to add code-gen support for LEFT ANTI sort merge join. The main change is to extract `loadStreamed` in `SortMergeJoinExec.doProduce()`. That is to set all columns values for streamed row, when the streamed row has no output row.

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"))
val df2 = spark.range(4).select($"id".as("k2"))
df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti")
```

Example generated code:

```
== Subtree 5 / 5 (maxMethodCodeSize:296; maxConstantPoolSize:156(0.24% used); numInnerClasses:0) ==
*(5) Project [id#0L AS k1#2L]
+- *(5) SortMergeJoin [id#0L], [k2#6L], LeftAnti
   :- *(2) Sort [id#0L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#0L, 5), ENSURE_REQUIREMENTS, [id=#27]
   :     +- *(1) Range (0, 10, step=1, splits=2)
   +- *(4) Sort [k2#6L ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(k2#6L, 5), ENSURE_REQUIREMENTS, [id=#33]
         +- *(3) Project [id#4L AS k2#6L]
            +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(1, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     smj_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */
/* 032 */   }
/* 033 */
/* 034 */   private boolean findNextJoinRows(
/* 035 */     scala.collection.Iterator streamedIter,
/* 036 */     scala.collection.Iterator bufferedIter) {
/* 037 */     smj_streamedRow_0 = null;
/* 038 */     int comp = 0;
/* 039 */     while (smj_streamedRow_0 == null) {
/* 040 */       if (!streamedIter.hasNext()) return false;
/* 041 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 042 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 043 */       if (false) {
/* 044 */         if (!smj_matches_0.isEmpty()) {
/* 045 */           smj_matches_0.clear();
/* 046 */         }
/* 047 */         return false;
/* 048 */
/* 049 */       }
/* 050 */       if (!smj_matches_0.isEmpty()) {
/* 051 */         comp = 0;
/* 052 */         if (comp == 0) {
/* 053 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 054 */         }
/* 055 */
/* 056 */         if (comp == 0) {
/* 057 */           return true;
/* 058 */         }
/* 059 */         smj_matches_0.clear();
/* 060 */       }
/* 061 */
/* 062 */       do {
/* 063 */         if (smj_bufferedRow_0 == null) {
/* 064 */           if (!bufferedIter.hasNext()) {
/* 065 */             smj_value_3 = smj_value_0;
/* 066 */             return !smj_matches_0.isEmpty();
/* 067 */           }
/* 068 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 069 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 070 */           if (false) {
/* 071 */             smj_bufferedRow_0 = null;
/* 072 */             continue;
/* 073 */           }
/* 074 */           smj_value_2 = smj_value_1;
/* 075 */         }
/* 076 */
/* 077 */         comp = 0;
/* 078 */         if (comp == 0) {
/* 079 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 080 */         }
/* 081 */
/* 082 */         if (comp > 0) {
/* 083 */           smj_bufferedRow_0 = null;
/* 084 */         } else if (comp < 0) {
/* 085 */           if (!smj_matches_0.isEmpty()) {
/* 086 */             smj_value_3 = smj_value_0;
/* 087 */             return true;
/* 088 */           } else {
/* 089 */             return false;
/* 090 */           }
/* 091 */         } else {
/* 092 */           if (smj_matches_0.isEmpty()) {
/* 093 */             smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 094 */           }
/* 095 */
/* 096 */           smj_bufferedRow_0 = null;
/* 097 */         }
/* 098 */       } while (smj_streamedRow_0 != null);
/* 099 */     }
/* 100 */     return false; // unreachable
/* 101 */   }
/* 102 */
/* 103 */   protected void processNext() throws java.io.IOException {
/* 104 */     while (smj_streamedInput_0.hasNext()) {
/* 105 */       findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0);
/* 106 */
/* 107 */       long smj_value_4 = -1L;
/* 108 */       smj_value_4 = smj_streamedRow_0.getLong(0);
/* 109 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 110 */
/* 111 */       boolean wholestagecodegen_hasOutputRow_0 = false;
/* 112 */
/* 113 */       while (!wholestagecodegen_hasOutputRow_0 && smj_iterator_0.hasNext()) {
/* 114 */         InternalRow smj_bufferedRow_1 = (InternalRow) smj_iterator_0.next();
/* 115 */
/* 116 */         wholestagecodegen_hasOutputRow_0 = true;
/* 117 */       }
/* 118 */
/* 119 */       if (!wholestagecodegen_hasOutputRow_0) {
/* 120 */         // load all values of streamed row, because the values not in join condition are not
/* 121 */         // loaded yet.
/* 122 */
/* 123 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 124 */
/* 125 */         // common sub-expressions
/* 126 */
/* 127 */         smj_mutableStateArray_0[1].reset();
/* 128 */
/* 129 */         smj_mutableStateArray_0[1].write(0, smj_value_4);
/* 130 */         append((smj_mutableStateArray_0[1].getRow()).copy());
/* 131 */
/* 132 */       }
/* 133 */       if (shouldStop()) return;
/* 134 */     }
/* 135 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 136 */   }
/* 137 */
/* 138 */ }
```

### Why are the changes needed?

Improve the query CPU performance.

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala`, and existed unit test in `ExistenceJoinSuite.scala`.

Closes #32547 from c21/smj-left-anti.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-18 16:56:45 +09:00
Takeshi Yamamuro 3b859a16c0 [SPARK-35431][SQL][TESTS] Sort elements generated by collect_set in SQLQueryTestSuite
### What changes were proposed in this pull request?

To pass `subquery/scalar-subquery/scalar-subquery-select.sql` (`SQLQueryTestSuite`) in Scala v2.13,  this PR proposes to change the aggregate expr of a test query in the file from `collect_set(...)` to `sort_array(collect_set(...))` because `collect_set` depends on the `mutable.HashSet` implementation and elements in the set are printed in a different order in Scala v2.12/v2.13.

### Why are the changes needed?

To pass the test in Scala v2.13.

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

No, dev-only.

### How was this patch tested?

Manually checked.

Closes #32578 from maropu/FixSQLTestIssueInScala213.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-17 22:51:32 -07:00
Chao Sun a60c36458d [SPARK-34981][SQL][TESTS][FOLLOWUP] Fix test failure under Scala 2.13
### What changes were proposed in this pull request?

Fix test failure under Scala 2.13 by making test `ScalaFunction` `StrLenMagic` public.

### Why are the changes needed?

A few tests are failing when using Scala 2.13 with error message like the following:
```
[info]   Cause: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 35, Column 121: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 35, Column 121: No a
pplicable constructor/method found for actual parameters "org.apache.spark.unsafe.types.UTF8String"; candidates are: "public int org.apache.spark.sql.connector.DataSourceV2FunctionSuite$StrLenMagic$.invoke(org.apache.spark.
unsafe.types.UTF8String)"
[info]   at org.apache.spark.sql.errors.QueryExecutionErrors$.compilerError(QueryExecutionErrors.scala:387)
[info]   at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1415)
[info]   at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1501)
```

This seems to be caused by the fact that the `StrLenMagic` is using `private` scope. After removing the `private` keyword the tests are now passing.

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

No

### How was this patch tested?

```
$ dev/change-scala-version.sh 2.13
$ build/sbt "sql/testOnly *.DataSourceV2FunctionSuite" -Pscala-2.13
```

Closes #32575 from sunchao/SPARK-34981-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-17 16:17:01 -07:00
fhygh 3a3f8ca6f4 [SPARK-35359][SQL] Insert data with char/varchar datatype will fail when data length exceed length limitation
### What changes were proposed in this pull request?
This PR is used to fix this bug:

```
set spark.sql.legacy.charVarcharAsString=true;
create table chartb01(a char(3));
insert into chartb01 select 'aaaaa';
```

here we expect the data of table chartb01 is 'aaa', but it runs failed.

### Why are the changes needed?
Improve backward compatibility

```
spark-sql>
         > create table tchar01(col char(2)) using parquet;
Time taken: 0.767 seconds
spark-sql>
         > insert into tchar01 select 'aaa';
ERROR | Executor task launch worker for task 0.0 in stage 0.0 (TID 0) | Aborting task | org.apache.spark.util.Utils.logError(Logging.scala:94)
java.lang.RuntimeException: Exceeds char/varchar type length limitation: 2
        at org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils.trimTrailingSpaces(CharVarcharCodegenUtils.java:31)
        at org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils.charTypeWriteSideCheck(CharVarcharCodegenUtils.java:44)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.project_doConsume_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:755)
        at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:279)
        at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1500)
        at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:288)
        at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:212)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:131)
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1466)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
        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)
```

### Does this PR introduce _any_ user-facing change?
No (the legacy config is false by default).

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

Closes #32501 from fhygh/master.

Authored-by: fhygh <283452027@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 00:13:40 +08:00
Wenchen Fan 3b63f32601 [SPARK-35400][SQL] Simplify getOuterReferences and improve error message for correlated subquery
### What changes were proposed in this pull request?

Spark doesn't support aggregate functions with mixed outer and local references. This PR applies this check earlier to fail with a clear error message instead of some weird ones, and simplifies the related code in `SubExprUtils.getOuterReferences`. This PR also refines the error message a bit.

### Why are the changes needed?

better error message

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

no

### How was this patch tested?

updated tests

Closes #32503 from cloud-fan/try.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-17 14:13:44 +00:00
Jungtaek Lim 7c13636be3 [SPARK-34888][SS] Introduce UpdatingSessionIterator adjusting session window on elements
Introduction: this PR is a part of SPARK-10816 (`EventTime based sessionization (session window)`). Please refer #31937 to see the overall view of the code change. (Note that code diff could be diverged a bit.)

### What changes were proposed in this pull request?

This PR introduces UpdatingSessionsIterator, which analyzes neighbor elements and adjust session information on elements.

UpdatingSessionsIterator calculates and updates the session window for each element in the given iterator, which makes elements in the same session window having same session spec. Downstream can apply aggregation to finally merge these elements bound to the same session window.

UpdatingSessionsIterator works on the precondition that given iterator is sorted by "group keys + start time of session window", and the iterator still retains the characteristic of the sort.

UpdatingSessionsIterator copies the elements to safely update on each element, as well as buffers elements which are bound to the same session window. Due to such overheads, MergingSessionsIterator which will be introduced via SPARK-34889 should be used whenever possible.

This PR also introduces UpdatingSessionsExec which is the physical node on leveraging UpdatingSessionsIterator to sort the input rows and updates session information on input rows.

### Why are the changes needed?

This part is a one of required on implementing SPARK-10816.

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

No.

### How was this patch tested?

New test suite added.

Closes #31986 from HeartSaVioR/SPARK-34888-SPARK-10816-PR-31570-part-1.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-17 21:05:49 +09:00
Yuming Wang fb9316388a [SPARK-32792][SQL][FOLLOWUP] Fix conflict with SPARK-34661
### What changes were proposed in this pull request?

This fixes the compilation error due to the logical conflicts between https://github.com/apache/spark/pull/31776 and https://github.com/apache/spark/pull/29642 .

### Why are the changes needed?

To recover compilation.

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

No.

### How was this patch tested?

Closes #32568 from wangyum/HOT-FIX.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 22:12:52 -07:00
Yuming Wang d2d1f0b580 [SPARK-32792][SQL] Improve Parquet In filter pushdown
### What changes were proposed in this pull request?

Support push down `GreaterThanOrEqual` minimum value and `LessThanOrEqual` maximum value for Parquet  when [sources.In](a744fea3be/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala (L162-L181))'s values exceeds `spark.sql.optimizer.inSetRewriteMinMaxThreshold`. For example:

```sql
SELECT * FROM t WHERE id IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 15)
```

We will push down `id >= 1 and id <= 15`.

Impala also has this improvement: https://issues.apache.org/jira/browse/IMPALA-3654

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test, [manual test](https://github.com/apache/spark/pull/29642#issuecomment-743109098) and benchmark test.

Before this PR:
```
================================================================================================
Pushdown benchmark for InSet -> InFilters
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5995           6026          53          2.6         381.2       1.0X
Parquet Vectorized (Pushdown)                                      423            440          11         37.2          26.9      14.2X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5767           5887         154          2.7         366.7       1.0X
Parquet Vectorized (Pushdown)                                      419            428           6         37.6          26.6      13.8X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5764           5857          96          2.7         366.4       1.0X
Parquet Vectorized (Pushdown)                                      408            419           9         38.6          25.9      14.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5895           5949          41          2.7         374.8       1.0X
Parquet Vectorized (Pushdown)                                      5908           5986         114          2.7         375.6       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5893           5988         106          2.7         374.7       1.0X
Parquet Vectorized (Pushdown)                                      5875           5939          57          2.7         373.5       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5891           5954          42          2.7         374.5       1.0X
Parquet Vectorized (Pushdown)                                      5901           5976          99          2.7         375.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6128           6158          40          2.6         389.6       1.0X
Parquet Vectorized (Pushdown)                                       6145           6190          37          2.6         390.7       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6142           6217          64          2.6         390.5       1.0X
Parquet Vectorized (Pushdown)                                       6149           6235          90          2.6         391.0       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6148           6218          64          2.6         390.9       1.0X
Parquet Vectorized (Pushdown)                                       6145           6177          30          2.6         390.7       1.0X
```

After this PR:
```
================================================================================================
Pushdown benchmark for InSet -> InFilters
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5745           5768          28          2.7         365.2       1.0X
Parquet Vectorized (Pushdown)                                      401            412          12         39.2          25.5      14.3X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5796           5861          61          2.7         368.5       1.0X
Parquet Vectorized (Pushdown)                                      417            482          37         37.7          26.5      13.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5754           5777          20          2.7         365.8       1.0X
Parquet Vectorized (Pushdown)                                      408            418           9         38.6          25.9      14.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5878           5915          40          2.7         373.7       1.0X
Parquet Vectorized (Pushdown)                                       929            940          10         16.9          59.1       6.3X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5886           5917          29          2.7         374.2       1.0X
Parquet Vectorized (Pushdown)                                      3091           3114          20          5.1         196.5       1.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5913           5948          48          2.7         375.9       1.0X
Parquet Vectorized (Pushdown)                                      5330           5427          98          3.0         338.9       1.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6147           6228          72          2.6         390.8       1.0X
Parquet Vectorized (Pushdown)                                       1023           1029           4         15.4          65.1       6.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6164           6224          47          2.6         391.9       1.0X
Parquet Vectorized (Pushdown)                                       3332           3360          45          4.7         211.9       1.8X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6154           6192          38          2.6         391.3       1.0X
Parquet Vectorized (Pushdown)                                       5588           5679          92          2.8         355.3       1.1X
```

Closes #29642 from wangyum/SPARK-32792.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <yumwang@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 21:20:52 -07:00
Takeshi Yamamuro 2eef2f9035 [SPARK-35412][SQL] Fix a bug in groupBy of year-month/day-time intervals
### What changes were proposed in this pull request?

To fix a bug below in groupBy of year-month/day-time intervals, this PR proposes to make `HashMapGenerator` handle the two types for hash-aggregates;
```
scala> Seq(java.time.Duration.ofDays(1)).toDF("a").groupBy("a").count().show()
scala.MatchError: DayTimeIntervalType (of class org.apache.spark.sql.types.DayTimeIntervalType$)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.genComputeHash(HashMapGenerator.scala:159)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.$anonfun$generateHashFunction$1(HashMapGenerator.scala:102)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.immutable.List.map(List.scala:298)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.genHashForKeys$1(HashMapGenerator.scala:99)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.generateHashFunction(HashMapGenerator.scala:111)
```

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Added a unit test.

Closes #32560 from maropu/FixIntervalIssue.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 10:51:32 -07:00
Cheng Su 5c1567ba97 [SPARK-35363][SQL][FOLLOWUP] Use fresh name for findNextJoinRows instead of hardcoding it
### What changes were proposed in this pull request?

This is a followup from discussion in https://github.com/apache/spark/pull/32495#discussion_r632283178 . The hardcoded function name `findNextJoinRows` is not a real problem now as we always do code generation for SMJ's children separately. But this change is to make it future proof in case this assumption changed in the future.

### Why are the changes needed?

Fix the potential reliability issue.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #32548 from c21/smj-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 10:49:31 -07:00
yangjie01 7ca0a0910f [SPARK-34661][SQL] Clean up OriginalType and DecimalMetadata usage in Parquet related code
### What changes were proposed in this pull request?
`OriginalType` and `DecimalMetadata` has been marked as `Deprecated` in new Parquet code.

`Apache Parquet` suggest us replace `OriginalType` with `LogicalTypeAnnotation` and replace `DecimalMetadata` with `DecimalLogicalTypeAnnotation`,  so the main change of this pr is clean up these deprecated usages in Parquet related code.

### Why are the changes needed?
Cleanup deprecated api usage.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31776 from LuciferYang/cleanup-parquet-dep-api.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-16 09:03:26 -05:00
QuangHuyViettel 9789ee84e4 [SPARK-32484][SQL] Fix log info BroadcastExchangeExec.scala
### What changes were proposed in this pull request?
Fix log info in BroadcastExchangeExec.scala

### Why are the changes needed?
Log info s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB")  is not accurate info , because  8GB  is not accurate.
### Does this PR introduce _any_ user-facing change?
yes

### How was this patch tested?
no

Closes #32544 from LittleCuteBug/SPARK-32484.

Authored-by: QuangHuyViettel <quanghuynguyen236@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-15 13:08:42 -05:00
Chao Sun a8032e7efa [SPARK-35384][SQL][FOLLOWUP] Move HashMap.get out of InvokeLike.invoke
### What changes were proposed in this pull request?

Move hash map lookup operation out of `InvokeLike.invoke` since it doesn't depend on the input.

### Why are the changes needed?

We shouldn't need to look up the hash map for every input row evaluated by `InvokeLike.invoke` since it doesn't depend on input. This could speed up the performance a bit.

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

No

### How was this patch tested?

Existing tests.

Closes #32532 from sunchao/SPARK-35384-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-14 14:00:39 -07:00
yi.wu 94bd480761 [SPARK-35206][TESTS][SQL] Extract common used get project path into a function in SparkFunctionSuite
### What changes were proposed in this pull request?

Add a common functions `getWorkspaceFilePath` (which prefixed with spark home) to `SparkFunctionSuite`, and applies these the function to where they're extracted from.

### Why are the changes needed?

Spark sql has test suites to read resources when running tests. The way of getting the path of resources is commonly used in different suites. We can extract them into a function to ease the code maintenance.

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

No.

### How was this patch tested?

Pass existing tests.

Closes #32315 from Ngone51/extract-common-file-path.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-14 22:17:50 +08:00
ulysses-you 6218bc5036 [SPARK-35332][SQL][FOLLOWUP] Refine wrong comment
### What changes were proposed in this pull request?

Refine comment in `CacheManager`.

### Why are the changes needed?

Avoid misleading developer.

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

No.

### How was this patch tested?

Not needed.

Closes #32543 from ulysses-you/SPARK-35332-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-14 17:10:21 +08:00
Takeshi Yamamuro 8fa739fb9d [SPARK-35329][SQL] Split generated switch code into pieces in ExpandExec
### What changes were proposed in this pull request?

This PR intends to split generated switch code into smaller ones in `ExpandExec`. In the current master, even a simple query like the one below generates a large method whose size (`maxMethodCodeSize:7448`) is close to `8000` (`CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT`);
```
scala> val df = Seq(("2016-03-27 19:39:34", 1, "a"), ("2016-03-27 19:39:56", 2, "a"), ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id")
scala> val rdf = df.select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value").orderBy($"window.start".asc, $"value".desc).select("value")
scala> sql("SET spark.sql.adaptive.enabled=false")
scala> import org.apache.spark.sql.execution.debug._
scala> rdf.debugCodegen

Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 (maxMethodCodeSize:7448; maxConstantPoolSize:189(0.29% used); numInnerClasses:0) ==
                                    ^^^^
*(1) Project [window#34.start AS _gen_alias_39#39, value#11]
+- *(1) Filter ((isnotnull(window#34) AND (cast(time#10 as timestamp) >= window#34.start)) AND (cast(time#10 as timestamp) < window#34.end))
   +- *(1) Expand [List(named_struct(start, precisetimestampcon...

/* 028 */   private void expand_doConsume_0(InternalRow localtablescan_row_0, UTF8String expand_expr_0_0, boolean expand_exprIsNull_0_0, int expand_expr_1_0) throws java.io.IOException {
/* 029 */     boolean expand_isNull_0 = true;
/* 030 */     InternalRow expand_value_0 =
/* 031 */     null;
/* 032 */     for (int expand_i_0 = 0; expand_i_0 < 4; expand_i_0 ++) {
/* 033 */       switch (expand_i_0) {
/* 034 */       case 0:
                  (too many code lines)
/* 517 */         break;
/* 518 */
/* 519 */       case 1:
                  (too many code lines)
/* 1002 */         break;
/* 1003 */
/* 1004 */       case 2:
                  (too many code lines)
/* 1487 */         break;
/* 1488 */
/* 1489 */       case 3:
                  (too many code lines)
/* 1972 */         break;
/* 1973 */       }
/* 1974 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[33] /* numOutputRows */).add(1);
/* 1975 */
/* 1976 */       do {
/* 1977 */         boolean filter_value_2 = !expand_isNull_0;
/* 1978 */         if (!filter_value_2) continue;
```
The fix in this PR can make the method smaller as follows;
```
Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 (maxMethodCodeSize:1713; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) ==
                                    ^^^^
*(1) Project [window#17.start AS _gen_alias_32#32, value#11]
+- *(1) Filter ((isnotnull(window#17) AND (cast(time#10 as timestamp) >= window#17.start)) AND (cast(time#10 as timestamp) < window#17.end))
   +- *(1) Expand [List(named_struct(start, precisetimestampcon...

/* 032 */   private void expand_doConsume_0(InternalRow localtablescan_row_0, UTF8String expand_expr_0_0, boolean expand_exprIsNull_0_0, int expand_expr_1_0) throws java.io.IOException {
/* 033 */     for (int expand_i_0 = 0; expand_i_0 < 4; expand_i_0 ++) {
/* 034 */       switch (expand_i_0) {
/* 035 */       case 0:
/* 036 */         expand_switchCaseCode_0(expand_exprIsNull_0_0, expand_expr_0_0);
/* 037 */         break;
/* 038 */
/* 039 */       case 1:
/* 040 */         expand_switchCaseCode_1(expand_exprIsNull_0_0, expand_expr_0_0);
/* 041 */         break;
/* 042 */
/* 043 */       case 2:
/* 044 */         expand_switchCaseCode_2(expand_exprIsNull_0_0, expand_expr_0_0);
/* 045 */         break;
/* 046 */
/* 047 */       case 3:
/* 048 */         expand_switchCaseCode_3(expand_exprIsNull_0_0, expand_expr_0_0);
/* 049 */         break;
/* 050 */       }
/* 051 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[33] /* numOutputRows */).add(1);
/* 052 */
/* 053 */       do {
/* 054 */         boolean filter_value_2 = !expand_resultIsNull_0;
/* 055 */         if (!filter_value_2) continue;
/* 056 */
...
```

### Why are the changes needed?

For better generated code.

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

No.

### How was this patch tested?

GA passed.

Closes #32457 from maropu/splitSwitchCode.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-13 17:53:46 -07:00
Linhong Liu 6aa2594c6b [SPARK-35366][SQL] Avoid using deprecated buildForBatch and buildForStreaming
### What changes were proposed in this pull request?
Currently, in DSv2, we are still using the deprecated `buildForBatch` and `buildForStreaming`.
This PR implements the `build`, `toBatch`, `toStreaming` interfaces to replace the deprecated ones.

### Why are the changes needed?
Code refactor

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

### How was this patch tested?
exsting UT

Closes #32497 from linhongliu-db/dsv2-writer.

Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 17:23:08 +00:00
gengjiaan c2e15cccab [SPARK-35062][SQL] Group exception messages in sql/streaming
### What changes were proposed in this pull request?
This PR group exception messages in `sql/core/src/main/scala/org/apache/spark/sql/streaming`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32464 from beliefer/SPARK-35062.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 15:04:03 +00:00
ulysses-you 6f63057ede [SPARK-35332][SQL] Make cache plan disable configs configurable
### What changes were proposed in this pull request?

Add a new config to make cache plan disable configs configurable.

### Why are the changes needed?

The disable configs of cache plan if to avoid the perfermance regression, but not all the query will slow than before due to AQE or bucket scan enabled. It's useful to make a new config so that user can decide if some configs should be disabled during cache plan.

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

Yes, a new config.

### How was this patch tested?

Add test.

Closes #32482 from ulysses-you/SPARK-35332.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 14:49:05 +00:00
Gengliang Wang 02c99f15ee [SPARK-35162][SQL] New SQL functions: TRY_ADD/TRY_DIVIDE
### What changes were proposed in this pull request?

Add New SQL functions:
* TRY_ADD
* TRY_DIVIDE

These expressions are identical to the following expression under ANSI mode except that it returns null if error occurs:
* ADD
* DIVIDE

Note: it is easy to add other expressions like `TRY_SUBTRACT`/`TRY_MULTIPLY` but let's control the number of these new expressions and just add `TRY_ADD` and `TRY_DIVIDE` for now.

### Why are the changes needed?

1. Users can manage to finish queries without interruptions in ANSI mode.
2. Users can get NULLs instead of unreasonable results if overflow occurs when ANSI mode is off.
For example, the behavior of the following SQL operations is unreasonable:
```
2147483647 + 2 => -2147483647
```

With the new safe version SQL functions:
```
TRY_ADD(2147483647, 2) => null
```

Note: **We should only add new expressions to important operators, instead of adding new safe expressions for all the expressions that can throw errors.**
### Does this PR introduce _any_ user-facing change?

Yes, new SQL functions: TRY_ADD/TRY_DIVIDE

### How was this patch tested?

Unit test

Closes #32292 from gengliangwang/try_add.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-13 22:26:08 +08:00
jiake b6d57b6b99 [SPARK-34637][SQL] Support DPP + AQE when the broadcast exchange can be reused
### What changes were proposed in this pull request?
We have supported DPP in AQE when the join is Broadcast hash join before applying the AQE rules in [SPARK-34168](https://issues.apache.org/jira/browse/SPARK-34168), which has some limitations. It only apply DPP when the small table side executed firstly and then the big table side can reuse the broadcast exchange in small table side. This PR is to address the above limitations and can apply the DPP when the broadcast exchange can be reused.

### Why are the changes needed?
Resolve the limitations when both enabling DPP and AQE

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

### How was this patch tested?
Adding new ut

Closes #31756 from JkSelf/supportDPP2.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 13:07:02 +00:00
Wenchen Fan d1b8bd7d11 [SPARK-34720][SQL] MERGE ... UPDATE/INSERT * should do by-name resolution
### What changes were proposed in this pull request?

In Spark, we have an extension in the MERGE syntax: INSERT/UPDATE *. This is not from ANSI standard or any other mainstream databases, so we need to define the behaviors by our own.

The behavior today is very weird: assume the source table has `n1` columns, target table has `n2` columns. We generate the assignments by taking the first `min(n1, n2)` columns from source & target tables and pairing them by ordinal.

This PR proposes a more reasonable behavior: take all the columns from target table as keys, and find the corresponding columns from source table by name as values.

### Why are the changes needed?

Fix the MEREG INSERT/UPDATE * to be more user-friendly and easy to do schema evolution.

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

Yes, but MERGE is only supported by very few data sources.

### How was this patch tested?

new tests

Closes #32192 from cloud-fan/merge.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 12:58:24 +00:00
Cheng Su c1e995ac95 [SPARK-35350][SQL] Add code-gen for left semi sort merge join
### What changes were proposed in this pull request?

As title. This PR is to add code-gen support for LEFT SEMI sort merge join. The main change is to add `semiJoin` code path in `SortMergeJoinExec.doProduce()` and introduce `onlyBufferFirstMatchedRow` in `SortMergeJoinExec.genScanner()`. The latter is for left semi sort merge join without condition. For this kind of query, we don't need to buffer all matched rows, but only the first one (this is same as non-code-gen code path).

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"))
val df2 = spark.range(4).select($"id".as("k2"))
val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi")
```

Example of generated code for the query:

```
== Subtree 5 / 5 (maxMethodCodeSize:302; maxConstantPoolSize:156(0.24% used); numInnerClasses:0) ==
*(5) Project [id#0L AS k1#2L]
+- *(5) SortMergeJoin [id#0L], [k2#6L], LeftSemi
   :- *(2) Sort [id#0L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#0L, 5), ENSURE_REQUIREMENTS, [id=#27]
   :     +- *(1) Range (0, 10, step=1, splits=2)
   +- *(4) Sort [k2#6L ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(k2#6L, 5), ENSURE_REQUIREMENTS, [id=#33]
         +- *(3) Project [id#4L AS k2#6L]
            +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(1, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     smj_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */
/* 032 */   }
/* 033 */
/* 034 */   private boolean findNextJoinRows(
/* 035 */     scala.collection.Iterator streamedIter,
/* 036 */     scala.collection.Iterator bufferedIter) {
/* 037 */     smj_streamedRow_0 = null;
/* 038 */     int comp = 0;
/* 039 */     while (smj_streamedRow_0 == null) {
/* 040 */       if (!streamedIter.hasNext()) return false;
/* 041 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 042 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 043 */       if (false) {
/* 044 */         smj_streamedRow_0 = null;
/* 045 */         continue;
/* 046 */
/* 047 */       }
/* 048 */       if (!smj_matches_0.isEmpty()) {
/* 049 */         comp = 0;
/* 050 */         if (comp == 0) {
/* 051 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 052 */         }
/* 053 */
/* 054 */         if (comp == 0) {
/* 055 */           return true;
/* 056 */         }
/* 057 */         smj_matches_0.clear();
/* 058 */       }
/* 059 */
/* 060 */       do {
/* 061 */         if (smj_bufferedRow_0 == null) {
/* 062 */           if (!bufferedIter.hasNext()) {
/* 063 */             smj_value_3 = smj_value_0;
/* 064 */             return !smj_matches_0.isEmpty();
/* 065 */           }
/* 066 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 067 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 068 */           if (false) {
/* 069 */             smj_bufferedRow_0 = null;
/* 070 */             continue;
/* 071 */           }
/* 072 */           smj_value_2 = smj_value_1;
/* 073 */         }
/* 074 */
/* 075 */         comp = 0;
/* 076 */         if (comp == 0) {
/* 077 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 078 */         }
/* 079 */
/* 080 */         if (comp > 0) {
/* 081 */           smj_bufferedRow_0 = null;
/* 082 */         } else if (comp < 0) {
/* 083 */           if (!smj_matches_0.isEmpty()) {
/* 084 */             smj_value_3 = smj_value_0;
/* 085 */             return true;
/* 086 */           } else {
/* 087 */             smj_streamedRow_0 = null;
/* 088 */           }
/* 089 */         } else {
/* 090 */           if (smj_matches_0.isEmpty()) {
/* 091 */             smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 092 */           }
/* 093 */
/* 094 */           smj_bufferedRow_0 = null;
/* 095 */         }
/* 096 */       } while (smj_streamedRow_0 != null);
/* 097 */     }
/* 098 */     return false; // unreachable
/* 099 */   }
/* 100 */
/* 101 */   protected void processNext() throws java.io.IOException {
/* 102 */     while (findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0)) {
/* 103 */       long smj_value_4 = -1L;
/* 104 */       smj_value_4 = smj_streamedRow_0.getLong(0);
/* 105 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 106 */       boolean smj_hasOutputRow_0 = false;
/* 107 */
/* 108 */       while (!smj_hasOutputRow_0 && smj_iterator_0.hasNext()) {
/* 109 */         InternalRow smj_bufferedRow_1 = (InternalRow) smj_iterator_0.next();
/* 110 */
/* 111 */         smj_hasOutputRow_0 = true;
/* 112 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 113 */
/* 114 */         // common sub-expressions
/* 115 */
/* 116 */         smj_mutableStateArray_0[1].reset();
/* 117 */
/* 118 */         smj_mutableStateArray_0[1].write(0, smj_value_4);
/* 119 */         append((smj_mutableStateArray_0[1].getRow()).copy());
/* 120 */
/* 121 */       }
/* 122 */       if (shouldStop()) return;
/* 123 */     }
/* 124 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 125 */   }
/* 126 */
/* 127 */ }
```

### Why are the changes needed?

Improve query CPU performance. Test with one query:

```
 def sortMergeJoin(): Unit = {
    val N = 2 << 20
    codegenBenchmark("left semi sort merge join", N) {
      val df1 = spark.range(N).selectExpr(s"id * 2 as k1")
      val df2 = spark.range(N).selectExpr(s"id * 3 as k2")
      val df = df1.join(df2, col("k1") === col("k2"), "left_semi")
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined)
      df.noop()
    }
  }
```

Seeing 30% of run-time improvement:

```
Running benchmark: left semi sort merge join
  Running case: left semi sort merge join code-gen off
  Stopped after 2 iterations, 1369 ms
  Running case: left semi sort merge join code-gen on
  Stopped after 5 iterations, 2743 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left semi sort merge join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
left semi sort merge join code-gen off              676            685          13          3.1         322.2       1.0X
left semi sort merge join code-gen on               524            549          32          4.0         249.7       1.3X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala` and `ExistenceJoinSuite.scala`.

Closes #32528 from c21/smj-left-semi.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 12:52:26 +00:00
Kent Yao 51815430b2 [SPARK-35380][SQL] Loading SparkSessionExtensions from ServiceLoader
### What changes were proposed in this pull request?

In https://github.com/yaooqinn/itachi/issues/8, we had a discussion about the current extension injection for the spark session.  We've agreed that the current way is not that convenient for both third-party developers and end-users.

It's much simple if third-party developers can provide a resource file that contains default extensions for Spark to  load ahead

### Why are the changes needed?

better use experience

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

no, dev change

### How was this patch tested?

new tests

Closes #32515 from yaooqinn/SPARK-35380.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-13 16:34:13 +08:00
Chao Sun 0ab9bd79b3 [SPARK-35384][SQL] Improve performance for InvokeLike.invoke
### What changes were proposed in this pull request?

Change `map` in `InvokeLike.invoke` to a while loop to improve performance, following Spark [style guide](https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex).

### Why are the changes needed?

`InvokeLike.invoke`, which is used in non-codegen path for `Invoke` and `StaticInvoke`, currently uses `map` to evaluate arguments:
```scala
val args = arguments.map(e => e.eval(input).asInstanceOf[Object])
if (needNullCheck && args.exists(_ == null)) {
  // return null if one of arguments is null
  null
} else {
  ...
```
which is pretty expensive if the method itself is trivial. We can change it to a plain while loop.

<img width="871" alt="Screen Shot 2021-05-12 at 12 19 59 AM" src="https://user-images.githubusercontent.com/506679/118055719-7f985a00-b33d-11eb-943b-cf85eab35f44.png">

Benchmark results show this can improve as much as 3x from `V2FunctionBenchmark`:

Before
```
 OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Linux 5.4.0-1046-azure
 Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
 scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 --------------------------------------------------------------------------------------------------------------------------------------------------------------
 native_long_add                                                                         36506          36656         251         13.7          73.0       1.0X
 java_long_add_default                                                                   47151          47540         370         10.6          94.3       0.8X
 java_long_add_magic                                                                    178691         182457        1327          2.8         357.4       0.2X
 java_long_add_static_magic                                                             177151         178258        1151          2.8         354.3       0.2X
```

After
```
 OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Linux 5.4.0-1046-azure
 Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
 scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 --------------------------------------------------------------------------------------------------------------------------------------------------------------
 native_long_add                                                                         29897          30342         568         16.7          59.8       1.0X
 java_long_add_default                                                                   40628          41075         664         12.3          81.3       0.7X
 java_long_add_magic                                                                     54553          54755         182          9.2         109.1       0.5X
 java_long_add_static_magic                                                              55410          55532         127          9.0         110.8       0.5X
```

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

No

### How was this patch tested?

Existing tests.

Closes #32527 from sunchao/SPARK-35384.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-12 20:57:21 -07:00
Takeshi Yamamuro 3241aeb7f4 [SPARK-35385][SQL][TESTS] Skip duplicate queries in the TPCDS-related tests
### What changes were proposed in this pull request?

This PR proposes to skip the "q6", "q34", "q64", "q74", "q75", "q78" queries in the TPCDS-related tests because the TPCDS v2.7 queries have almost the same ones; the only differences in these queries are ORDER BY columns.

### Why are the changes needed?

To improve test performance.

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

No, dev only.

### How was this patch tested?

Existing tests.

Closes #32520 from maropu/SkipDupQueries.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-13 09:46:25 +09:00
Chao Sun bc95c3a69b [SPARK-35361][SQL][FOLLOWUP] Switch to use while loop
### What changes were proposed in this pull request?

Switch to plain `while` loop following Spark [style guide](https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex).

### Why are the changes needed?

`while` loop may yield better performance comparing to `foreach`.

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

No

### How was this patch tested?

N/A

Closes #32522 from sunchao/SPARK-35361-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 12:41:12 -07:00
Cheng Su 7bcadedbd2 [SPARK-35349][SQL] Add code-gen for left/right outer sort merge join
### What changes were proposed in this pull request?

This PR is to add code-gen support for LEFT OUTER / RIGHT OUTER sort merge join. Currently sort merge join only supports inner join type (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala#L374 ). There's no fundamental reason why we cannot support code-gen for other join types. Here we add code-gen for LEFT OUTER / RIGHT OUTER join. Will submit followup PRs to add LEFT SEMI, LEFT ANTI and FULL OUTER code-gen separately.

The change is to extend current sort merge join logic to work with LEFT OUTER and RIGHT OUTER (should work with LEFT SEMI/ANTI as well, but FULL OUTER join needs some other more code change). Replace left/right with streamed/buffered to make code extendable to other join types besides inner join.

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"), $"id".as("k3"))
val df2 = spark.range(4).select($"id".as("k2"), $"id".as("k4"))
df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2" && $"k3" + 1 < $"k4", "left_outer").explain("codegen")
```

Example generated code:

```
== Subtree 5 / 5 (maxMethodCodeSize:396; maxConstantPoolSize:159(0.24% used); numInnerClasses:0) ==
*(5) SortMergeJoin [k1#2L], [k2#8L], LeftOuter, ((k3#3L + 1) < k4#9L)
:- *(2) Sort [k1#2L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(k1#2L, 5), ENSURE_REQUIREMENTS, [id=#26]
:     +- *(1) Project [id#0L AS k1#2L, id#0L AS k3#3L]
:        +- *(1) Range (0, 10, step=1, splits=2)
+- *(4) Sort [k2#8L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k2#8L, 5), ENSURE_REQUIREMENTS, [id=#32]
      +- *(3) Project [id#6L AS k2#8L, id#6L AS k4#9L]
         +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(2147483632, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(4, 0);
/* 030 */
/* 031 */   }
/* 032 */
/* 033 */   private boolean findNextJoinRows(
/* 034 */     scala.collection.Iterator streamedIter,
/* 035 */     scala.collection.Iterator bufferedIter) {
/* 036 */     smj_streamedRow_0 = null;
/* 037 */     int comp = 0;
/* 038 */     while (smj_streamedRow_0 == null) {
/* 039 */       if (!streamedIter.hasNext()) return false;
/* 040 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 041 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 042 */       if (false) {
/* 043 */         if (!smj_matches_0.isEmpty()) {
/* 044 */           smj_matches_0.clear();
/* 045 */         }
/* 046 */         return false;
/* 047 */
/* 048 */       }
/* 049 */       if (!smj_matches_0.isEmpty()) {
/* 050 */         comp = 0;
/* 051 */         if (comp == 0) {
/* 052 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 053 */         }
/* 054 */
/* 055 */         if (comp == 0) {
/* 056 */           return true;
/* 057 */         }
/* 058 */         smj_matches_0.clear();
/* 059 */       }
/* 060 */
/* 061 */       do {
/* 062 */         if (smj_bufferedRow_0 == null) {
/* 063 */           if (!bufferedIter.hasNext()) {
/* 064 */             smj_value_3 = smj_value_0;
/* 065 */             return !smj_matches_0.isEmpty();
/* 066 */           }
/* 067 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 068 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 069 */           if (false) {
/* 070 */             smj_bufferedRow_0 = null;
/* 071 */             continue;
/* 072 */           }
/* 073 */           smj_value_2 = smj_value_1;
/* 074 */         }
/* 075 */
/* 076 */         comp = 0;
/* 077 */         if (comp == 0) {
/* 078 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 079 */         }
/* 080 */
/* 081 */         if (comp > 0) {
/* 082 */           smj_bufferedRow_0 = null;
/* 083 */         } else if (comp < 0) {
/* 084 */           if (!smj_matches_0.isEmpty()) {
/* 085 */             smj_value_3 = smj_value_0;
/* 086 */             return true;
/* 087 */           } else {
/* 088 */             return false;
/* 089 */           }
/* 090 */         } else {
/* 091 */           smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 092 */           smj_bufferedRow_0 = null;
/* 093 */         }
/* 094 */       } while (smj_streamedRow_0 != null);
/* 095 */     }
/* 096 */     return false; // unreachable
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     while (smj_streamedInput_0.hasNext()) {
/* 101 */       findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0);
/* 102 */       long smj_value_4 = -1L;
/* 103 */       long smj_value_5 = -1L;
/* 104 */       boolean smj_loaded_0 = false;
/* 105 */       smj_value_5 = smj_streamedRow_0.getLong(1);
/* 106 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 107 */       boolean smj_foundMatch_0 = false;
/* 108 */
/* 109 */       // the last iteration of this loop is to emit an empty row if there is no matched rows.
/* 110 */       while (smj_iterator_0.hasNext() || !smj_foundMatch_0) {
/* 111 */         InternalRow smj_bufferedRow_1 = smj_iterator_0.hasNext() ?
/* 112 */         (InternalRow) smj_iterator_0.next() : null;
/* 113 */         boolean smj_isNull_5 = true;
/* 114 */         long smj_value_9 = -1L;
/* 115 */         if (smj_bufferedRow_1 != null) {
/* 116 */           long smj_value_8 = smj_bufferedRow_1.getLong(1);
/* 117 */           smj_isNull_5 = false;
/* 118 */           smj_value_9 = smj_value_8;
/* 119 */         }
/* 120 */         if (smj_bufferedRow_1 != null) {
/* 121 */           boolean smj_isNull_6 = true;
/* 122 */           boolean smj_value_10 = false;
/* 123 */           long smj_value_11 = -1L;
/* 124 */
/* 125 */           smj_value_11 = smj_value_5 + 1L;
/* 126 */
/* 127 */           if (!smj_isNull_5) {
/* 128 */             smj_isNull_6 = false; // resultCode could change nullability.
/* 129 */             smj_value_10 = smj_value_11 < smj_value_9;
/* 130 */
/* 131 */           }
/* 132 */           if (smj_isNull_6 || !smj_value_10) {
/* 133 */             continue;
/* 134 */           }
/* 135 */         }
/* 136 */         if (!smj_loaded_0) {
/* 137 */           smj_loaded_0 = true;
/* 138 */           smj_value_4 = smj_streamedRow_0.getLong(0);
/* 139 */         }
/* 140 */         boolean smj_isNull_3 = true;
/* 141 */         long smj_value_7 = -1L;
/* 142 */         if (smj_bufferedRow_1 != null) {
/* 143 */           long smj_value_6 = smj_bufferedRow_1.getLong(0);
/* 144 */           smj_isNull_3 = false;
/* 145 */           smj_value_7 = smj_value_6;
/* 146 */         }
/* 147 */         smj_foundMatch_0 = true;
/* 148 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 149 */
/* 150 */         smj_mutableStateArray_0[0].reset();
/* 151 */
/* 152 */         smj_mutableStateArray_0[0].zeroOutNullBytes();
/* 153 */
/* 154 */         smj_mutableStateArray_0[0].write(0, smj_value_4);
/* 155 */
/* 156 */         smj_mutableStateArray_0[0].write(1, smj_value_5);
/* 157 */
/* 158 */         if (smj_isNull_3) {
/* 159 */           smj_mutableStateArray_0[0].setNullAt(2);
/* 160 */         } else {
/* 161 */           smj_mutableStateArray_0[0].write(2, smj_value_7);
/* 162 */         }
/* 163 */
/* 164 */         if (smj_isNull_5) {
/* 165 */           smj_mutableStateArray_0[0].setNullAt(3);
/* 166 */         } else {
/* 167 */           smj_mutableStateArray_0[0].write(3, smj_value_9);
/* 168 */         }
/* 169 */         append((smj_mutableStateArray_0[0].getRow()).copy());
/* 170 */
/* 171 */       }
/* 172 */       if (shouldStop()) return;
/* 173 */     }
/* 174 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 175 */   }
/* 176 */
/* 177 */ }
```

### Why are the changes needed?

Improve query CPU performance. Example micro benchmark below showed 10% run-time improvement.

```
def sortMergeJoinWithDuplicates(): Unit = {
    val N = 2 << 20
    codegenBenchmark("sort merge join with duplicates", N) {
      val df1 = spark.range(N)
        .selectExpr(s"(id * 15485863) % ${N*10} as k1", "id as k3")
      val df2 = spark.range(N)
        .selectExpr(s"(id * 15485867) % ${N*10} as k2", "id as k4")
      val df = df1.join(df2, col("k1") === col("k2") && col("k3") * 3 < col("k4"), "left_outer")
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined)
      df.noop()
    }
 }
```

```
Running benchmark: sort merge join with duplicates
  Running case: sort merge join with duplicates outer-smj-codegen off
  Stopped after 2 iterations, 2696 ms
  Running case: sort merge join with duplicates outer-smj-codegen on
  Stopped after 5 iterations, 6058 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
sort merge join with duplicates:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates outer-smj-codegen off           1333           1348          21          1.6         635.7       1.0X
sort merge join with duplicates outer-smj-codegen on            1169           1212          47          1.8         557.4       1.1X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala` and `WholeStageCodegenSuite.scala`.

Closes #32476 from c21/smj-outer-codegen.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-12 14:10:15 +00:00
PengLei 82c520a3e2 [SPARK-35243][SQL] Support columnar execution on ANSI interval types
### What changes were proposed in this pull request?
Columnar execution support for ANSI interval types include YearMonthIntervalType and DayTimeIntervalType

### Why are the changes needed?
support cache tables with ANSI interval types.

### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
run ./dev/lint-java
run ./dev/scalastyle
run test: CachedTableSuite
run test: ColumnTypeSuite

Closes #32452 from Peng-Lei/SPARK-35243.

Lead-authored-by: PengLei <18066542445@189.cn>
Co-authored-by: Lei Peng <peng.8lei@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 20:11:34 +09:00
Chao Sun 78221bda95 [SPARK-35361][SQL] Improve performance for ApplyFunctionExpression
### What changes were proposed in this pull request?

In `ApplyFunctionExpression`, move `zipWithIndex` out of the loop for each input row.

### Why are the changes needed?

When the `ScalarFunction` is trivial, `zipWithIndex` could incur significant costs, as shown below:

<img width="899" alt="Screen Shot 2021-05-11 at 10 03 42 AM" src="https://user-images.githubusercontent.com/506679/117866421-fb19de80-b24b-11eb-8c94-d5e8c8b1eda9.png">

By removing it out of the loop, I'm seeing sometimes 2x speedup from `V2FunctionBenchmark`. For instance:

Before:
```
scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
native_long_add                                                                         32437          32896         434         15.4          64.9       1.0X
java_long_add_default                                                                   85675          97045         NaN          5.8         171.3       0.4X
```

After:
```
scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
native_long_add                                                                         30182          30387         279         16.6          60.4       1.0X
java_long_add_default                                                                   42862          43009         209         11.7          85.7       0.7X
```

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

No

### How was this patch tested?

Existing tests

Closes #32507 from sunchao/SPARK-35361.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 10:16:35 +09:00
Cheng Su c4ca23207b [SPARK-35363][SQL] Refactor sort merge join code-gen be agnostic to join type
### What changes were proposed in this pull request?

This is a pre-requisite of https://github.com/apache/spark/pull/32476, in discussion of https://github.com/apache/spark/pull/32476#issuecomment-836469779 . This is to refactor sort merge join code-gen to depend on streamed/buffered terminology, which makes the code-gen agnostic to different join types and can be extended to support other join types than inner join.

### Why are the changes needed?

Pre-requisite of https://github.com/apache/spark/pull/32476.

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

No.

### How was this patch tested?

Existing unit test in `InnerJoinSuite.scala` for inner join code-gen.

Closes #32495 from c21/smj-refactor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-11 11:21:59 +09:00
Angerszhuuuu 7182f8cece [SPARK-35360][SQL] RepairTableCommand respects spark.sql.addPartitionInBatch.size too
### What changes were proposed in this pull request?
RepairTableCommand respects `spark.sql.addPartitionInBatch.size` too

### Why are the changes needed?
Make RepairTableCommand add partition batch size configurable.

### Does this PR introduce _any_ user-facing change?
User can use `spark.sql.addPartitionInBatch.size` to change batch size when repair table.

### How was this patch tested?
Not need

Closes #32489 from AngersZhuuuu/SPARK-35360.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-10 14:53:31 +05:00
Chao Sun 245dce1ea1 [SPARK-35261][SQL][TESTS][FOLLOW-UP] Change failOnError to false for NativeAdd in V2FunctionBenchmark
### What changes were proposed in this pull request?

Change `failOnError` to false for `NativeAdd` in `V2FunctionBenchmark`.

### Why are the changes needed?

Since `NativeAdd` is simply doing addition on long it's better to set `failOnError` to false so it will use native long addition instead of `Math.addExact`.

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

No

### How was this patch tested?

N/A

Closes #32481 from sunchao/SPARK-35261-follow-up.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-10 07:20:05 +00:00
Cheng Su 38eb5a6936 [SPARK-35354][SQL] Replace BaseJoinExec with ShuffledJoin in CoalesceBucketsInJoin
### What changes were proposed in this pull request?

As title. We should use a more restrictive interface `ShuffledJoin` other than `BaseJoinExec` in `CoalesceBucketsInJoin`, as the rule only applies to sort merge join and shuffled hash join (i.e. `ShuffledJoin`).

### Why are the changes needed?

Code cleanup.

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

No.

### How was this patch tested?

Existing unit test in `CoalesceBucketsInJoinSuite`.

Closes #32480 from c21/minor-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-10 10:04:49 +09:00
Takeshi Yamamuro 06c40091a6 [SPARK-35327][SQL][TESTS] Filters out the TPC-DS queries that can cause flaky test results
### What changes were proposed in this pull request?

This PR proposes to filter out TPCDS v1.4 q6 and q75 in `TPCDSQueryTestSuite`.

I saw`TPCDSQueryTestSuite` failed nondeterministically because output row orders were different with those in the golden files. For example, the failure in the GA job, https://github.com/linhongliu-db/spark/runs/2507928605?check_suite_focus=true, happened because the `tpcds/q6.sql` query output rows were only sorted by `cnt`:

a0c76a8755/sql/core/src/test/resources/tpcds/q6.sql (L20)
Actually, `tpcds/q6.sql`  and `tpcds-v2.7.0/q6.sql` are almost the same and the only difference is that `tpcds-v2.7.0/q6.sql` sorts both `cnt` and `a.ca_state`:
a0c76a8755/sql/core/src/test/resources/tpcds-v2.7.0/q6.sql (L22)
So, I think it's okay just to test `tpcds-v2.7.0/q6.sql` in this case (q75 has the same issue).

### Why are the changes needed?

For stable testing.

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

No, dev-only.

### How was this patch tested?

GA passed.

Closes #32454 from maropu/CleanUpTpcdsQueries.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-08 21:43:39 +09:00
Kent Yao b0257801d5 [SPARK-35331][SQL] Support resolving missing attrs for distribute/cluster by/repartition hint
### What changes were proposed in this pull request?

This PR makes the below case work well.

```sql
select a b from values(1) t(a) distribute by a;
```

```logtalk
== Parsed Logical Plan ==
'RepartitionByExpression ['a]
+- 'Project ['a AS b#42]
   +- 'SubqueryAlias t
      +- 'UnresolvedInlineTable [a], [List(1)]

== Analyzed Logical Plan ==
org.apache.spark.sql.AnalysisException: cannot resolve 'a' given input columns: [b]; line 1 pos 62;
'RepartitionByExpression ['a]
+- Project [a#48 AS b#42]
   +- SubqueryAlias t
      +- LocalRelation [a#48]
```
### Why are the changes needed?

bugfix

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

yes, the original attributes can be used in `distribute by` / `cluster by` and hints like `/*+ REPARTITION(3, c) */`

### How was this patch tested?

new tests

Closes #32465 from yaooqinn/SPARK-35331.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-08 05:00:51 -07:00
Chao Sun f47e0f8379 [SPARK-35261][SQL] Support static magic method for stateless Java ScalarFunction
### What changes were proposed in this pull request?

This allows `ScalarFunction` implemented in Java to optionally specify the magic method `invoke` to be static, which can be used if the UDF is stateless. Comparing to the non-static method, it can potentially give better performance due to elimination of dynamic dispatch, etc.

Also added a benchmark to measure performance of: the default `produceResult`, non-static magic method and static magic method.

### Why are the changes needed?

For UDFs that are stateless (e.g., no need to maintain intermediate state between each function call), it's better to allow users to implement the UDF function as static method which could potentially give better performance.

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

Yes. Spark users can now have the choice to define static magic method for `ScalarFunction` when it is written in Java and when the UDF is stateless.

### How was this patch tested?

Added new UT.

Closes #32407 from sunchao/SPARK-35261.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 20:34:51 -07:00
Wenchen Fan e83910f1f8 [SPARK-26164][SQL][FOLLOWUP] WriteTaskStatsTracker should know which file the row is written to
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/32198

Before https://github.com/apache/spark/pull/32198, in `WriteTaskStatsTracker.newRow`, we know that the row is written to the current file. After https://github.com/apache/spark/pull/32198 , we no longer know this connection.

This PR adds the file path parameter in `WriteTaskStatsTracker.newRow` to bring back the connection.

### Why are the changes needed?

To not break some custom `WriteTaskStatsTracker` implementations.

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

no

### How was this patch tested?

N/A

Closes #32459 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 08:28:42 +00:00
Terry Kim 33c1034315 [SPARK-34701][SQL][FOLLOW-UP] Children/innerChildren should be mutually exclusive for AnalysisOnlyCommand
### What changes were proposed in this pull request?

This is a follow up to https://github.com/apache/spark/pull/32032#discussion_r620928086. Basically, `children`/`innerChildren` should be mutually exclusive for `AlterViewAsCommand` and `CreateViewCommand`, which extend `AnalysisOnlyCommand`. Otherwise, there could be an issue in the `EXPLAIN` command. Currently, this is not an issue, because these commands will be analyzed (children will always be empty) when the `EXPLAIN` command is run.

### Why are the changes needed?

To be future-proof where these commands are directly used.

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

No.

### How was this patch tested?

Added new tsts

Closes #32447 from imback82/SPARK-34701-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 06:07:53 +00:00
Cheng Su 42f59caf73 [SPARK-35133][SQL] Explain codegen works with AQE
### What changes were proposed in this pull request?

`EXPLAIN CODEGEN <query>` (and Dataset.explain("codegen")) prints out the generated code for each stage of plan. The current implementation is to match `WholeStageCodegenExec` operator in query plan and prints out generated code (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala#L111-L118 ). This does not work with AQE as we wrap the whole query plan inside `AdaptiveSparkPlanExec` and do not run whole stage code-gen physical plan rule eagerly (`CollapseCodegenStages`). This introduces unexpected behavior change for EXPLAIN query (and Dataset.explain), as we enable AQE by default now.

The change is to explain code-gen for the current executed plan of AQE.

### Why are the changes needed?

Make `EXPLAIN CODEGEN` work same as before.

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

No (when comparing with latest Spark release 3.1.1).

### How was this patch tested?

Added unit test in `ExplainSuite.scala`.

Closes #32430 from c21/explain-aqe.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-06 20:44:31 -07:00
Yuanjian Li dfb3343423 [SPARK-34526][SS] Ignore the error when checking the path in FileStreamSink.hasMetadata
### What changes were proposed in this pull request?
When checking the path in `FileStreamSink.hasMetadata`, we should ignore the error and assume the user wants to read a batch output.

### Why are the changes needed?
Keep the original behavior of ignoring the error.

### Does this PR introduce _any_ user-facing change?
Yes.
The path checking will not throw an exception when checking file sink format

### How was this patch tested?
New UT added.

Closes #31638 from xuanyuanking/SPARK-34526.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-06 22:48:53 +09:00
Liang-Chi Hsieh 6cd5cf5722 [SPARK-35215][SQL] Update custom metric per certain rows and at the end of the task
### What changes were proposed in this pull request?

This patch changes custom metric updating to update per certain rows (currently 100), instead of per row.

### Why are the changes needed?

Based on previous discussion https://github.com/apache/spark/pull/31451#discussion_r605413557, we should only update custom metrics per certain (e.g. 100) rows and also at the end of the task. Updating per row doesn't make too much benefit.

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

No

### How was this patch tested?

Existing unit test.

Closes #32330 from viirya/metric-update.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-06 13:21:08 +00:00
Liang-Chi Hsieh c6d3f3778f [SPARK-35240][SS] Use CheckpointFileManager for checkpoint file manipulation
### What changes were proposed in this pull request?

This patch changes a few places using `FileSystem` API to manipulate checkpoint file to `CheckpointFileManager`.

### Why are the changes needed?

`CheckpointFileManager` is designed to handle checkpoint file manipulation. However, there are a few places exposing `FileSystem` from checkpoint files/paths. We should use `CheckpointFileManager` to manipulate checkpoint files. For example, we may want to have one storage system for checkpoint file. If all checkpoint file manipulation is performed through `CheckpointFileManager`, we can only implement `CheckpointFileManager` for the storage system, and don't need to implement `FileSystem` API for it.

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

No

### How was this patch tested?

Existing unit tests.

Closes #32361 from viirya/checkpoint-manager.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-06 00:49:37 -07:00
Linhong Liu 3f5a20919c [SPARK-35318][SQL] Hide internal view properties for describe table cmd
### What changes were proposed in this pull request?
Hide internal view properties for describe table command, because those
properties are generated by spark and should be transparent to the end-user.

### Why are the changes needed?
Avoid internal properties confusing the users.

### Does this PR introduce _any_ user-facing change?
Yes
Before this change, the user will see below output for `describe formatted test_view`
```
....
Table Properties       [view.catalogAndNamespace.numParts=2, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=default, view.query.out.col.0=c, view.query.out.col.1=v, view.query.out.numCols=2, view.referredTempFunctionsNames=[], view.referredTempViewNames=[]]
...
```
After this change, the internal properties will be hidden for `describe formatted test_view`
```
...
Table Properties        []
...
```

### How was this patch tested?
existing UT

Closes #32441 from linhongliu-db/hide-properties.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-06 07:31:34 +00:00
Takeshi Yamamuro 5c67d0c8f7 [SPARK-35293][SQL][TESTS] Use the newer dsdgen for TPCDSQueryTestSuite
### What changes were proposed in this pull request?

This PR intends to replace `maropu/spark-tpcds-datagen` with `databricks/tpcds-kit` for using a newer dsdgen and update the golden files in `tpcds-query-results`.

### Why are the changes needed?

For better testing.

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

No.

### How was this patch tested?

GA passed.

Closes #32420 from maropu/UseTpcdsKit.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-06 15:25:46 +09:00
Dongjoon Hyun 19661f6ae2 [SPARK-35325][SQL][TESTS] Add nested column ORC encryption test case
### What changes were proposed in this pull request?

This PR aims to enrich ORC encryption test coverage for nested columns.

### Why are the changes needed?

This will provide a test coverage for this feature.

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

No.

### How was this patch tested?

Pass the CIs with the newly added test case.

Closes #32449 from dongjoon-hyun/SPARK-35325.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-05 22:29:54 -07:00
Yijia Cui bbdbe0f734 [SPARK-34854][SQL][SS] Expose source metrics via progress report and add Kafka use-case to report delay
### What changes were proposed in this pull request?
This pull request proposes a new API for streaming sources to signal that they can report metrics, and adds a use case to support Kafka micro batch stream to report the stats of # of offsets for the current offset falling behind the latest.

A public interface is added.

`metrics`: returns the metrics reported by the streaming source with given offset.

### Why are the changes needed?
The new API can expose any custom metrics for the "current" offset for streaming sources. Different from #31398, this PR makes metrics available to user through progress report, not through spark UI. A use case is that people want to know how the current offset falls behind the latest offset.

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

### How was this patch tested?
Unit test for Kafka micro batch source v2 are added to test the Kafka use case.

Closes #31944 from yijiacui-db/SPARK-34297.

Authored-by: Yijia Cui <yijia.cui@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-05 17:26:07 +09:00
dsolow f550e03b96 [SPARK-34794][SQL] Fix lambda variable name issues in nested DataFrame functions
### What changes were proposed in this pull request?

To fix lambda variable name issues in nested DataFrame functions, this PR modifies code to use a global counter for `LambdaVariables` names created by higher order functions.

This is the rework of #31887. Closes #31887.

### Why are the changes needed?

 This moves away from the current hard-coded variable names which break on nested function calls. There is currently a bug where nested transforms in particular fail (the inner variable shadows the outer variable)

For this query:
```
val df = Seq(
    (Seq(1,2,3), Seq("a", "b", "c"))
).toDF("numbers", "letters")

df.select(
    f.flatten(
        f.transform(
            $"numbers",
            (number: Column) => { f.transform(
                $"letters",
                (letter: Column) => { f.struct(
                    number.as("number"),
                    letter.as("letter")
                ) }
            ) }
        )
    ).as("zipped")
).show(10, false)
```
This is the current (incorrect) output:
```
+------------------------------------------------------------------------+
|zipped                                                                  |
+------------------------------------------------------------------------+
|[{a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}]|
+------------------------------------------------------------------------+
```
And this is the correct output after fix:
```
+------------------------------------------------------------------------+
|zipped                                                                  |
+------------------------------------------------------------------------+
|[{1, a}, {1, b}, {1, c}, {2, a}, {2, b}, {2, c}, {3, a}, {3, b}, {3, c}]|
+------------------------------------------------------------------------+
```

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

No

### How was this patch tested?

Added the new test in `DataFrameFunctionsSuite`.

Closes #32424 from maropu/pr31887.

Lead-authored-by: dsolow <dsolow@sayari.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: dmsolow <dsolow@sayarianalytics.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-05 12:46:13 +09:00
HyukjinKwon 8aaa9e890a [SPARK-35250][SQL][DOCS] Fix duplicated STOP_AT_DELIMITER to SKIP_VALUE at CSV's unescapedQuoteHandling option documentation
### What changes were proposed in this pull request?

This is rather a followup of https://github.com/apache/spark/pull/30518 that should be ported back to `branch-3.1` too.
`STOP_AT_DELIMITER` was mistakenly used twice. The duplicated `STOP_AT_DELIMITER` should be `SKIP_VALUE` in the documentation.

### Why are the changes needed?

To correctly document.

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

Yes, it fixes the user-facing documentation.

### How was this patch tested?

I checked them via running linters.

Closes #32423 from HyukjinKwon/SPARK-35250.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-04 08:44:18 +09:00
Tobias Hermann 54e0aa10c8 [MINOR][SS][DOCS] Fix a typo in the documentation of GroupState
### What changes were proposed in this pull request?

Fixing some typos in the documenting comments.

### Why are the changes needed?

To make reading the docs more pleasant.

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

Yes, since the user sees the docs.

### How was this patch tested?

It was not tested, because no code was changed.

Closes #32400 from Dobiasd/patch-1.

Authored-by: Tobias Hermann <editgym@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-03 19:35:38 +09:00
Max Gekk 335f00b19b [SPARK-35285][SQL] Parse ANSI interval types in SQL schema
### What changes were proposed in this pull request?
1. Extend Spark SQL parser to support parsing of:
    - `INTERVAL YEAR TO MONTH` to `YearMonthIntervalType`
    - `INTERVAL DAY TO SECOND` to `DayTimeIntervalType`
2. Assign new names to the ANSI interval types according to the SQL standard to be able to parse the names back by Spark SQL parser. Override the `typeName()` name of `YearMonthIntervalType`/`DayTimeIntervalType`.

### Why are the changes needed?
To be able to use new ANSI interval types in SQL. The SQL standard requires the types to be defined according to the rules:
```
<interval type> ::= INTERVAL <interval qualifier>
<interval qualifier> ::= <start field> TO <end field> | <single datetime field>
<start field> ::= <non-second primary datetime field> [ <left paren> <interval leading field precision> <right paren> ]
<end field> ::= <non-second primary datetime field> | SECOND [ <left paren> <interval fractional seconds precision> <right paren> ]
<primary datetime field> ::= <non-second primary datetime field | SECOND
<non-second primary datetime field> ::= YEAR | MONTH | DAY | HOUR | MINUTE
<interval fractional seconds precision> ::= <unsigned integer>
<interval leading field precision> ::= <unsigned integer>
```
Currently, Spark SQL supports only `YEAR TO MONTH` and `DAY TO SECOND` as `<interval qualifier>`.

### Does this PR introduce _any_ user-facing change?
Should not since the types has not been released yet.

### How was this patch tested?
By running the affected tests such as:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "test:testOnly *ExpressionTypeCheckingSuite"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z windowFrameCoercion.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z literals.sql"
```

Closes #32409 from MaxGekk/parse-ansi-interval-types.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-03 13:50:35 +09:00
Takeshi Yamamuro cd689c942c [SPARK-35192][SQL][TESTS] Port minimal TPC-DS datagen code from databricks/spark-sql-perf
### What changes were proposed in this pull request?

This PR proposes to port minimal code to generate TPC-DS data from [databricks/spark-sql-perf](https://github.com/databricks/spark-sql-perf). The classes in a new class file `tpcdsDatagen.scala` are basically copied from the `databricks/spark-sql-perf` codebase.
Note that I've modified them a bit to follow the Spark code style and removed unnecessary parts from them.

The code authors of these classes are:
juliuszsompolski
npoggi
wangyum

### Why are the changes needed?

We frequently use TPCDS data now for benchmarks/tests, but the classes for the TPCDS schemas of datagen and benchmarks/tests are managed separately, e.g.,
 - https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala
 - https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/sql/perf/tpcds/TPCDSTables.scala

 I think this causes some inconveniences, e.g., we need to update both files in the separate repositories if we update the TPCDS schema #32037. So, it would be useful for the Spark codebase to generate them by referring to the same schema definition.

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

dev only.

### How was this patch tested?

Manually checked and GA passed.

Closes #32243 from maropu/tpcdsDatagen.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-03 12:04:42 +09:00
Peter Toth cfc0495f9c [SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function
### What changes were proposed in this pull request?
This PR adds a new rule `PullOutGroupingExpressions` to pull out complex grouping expressions to a `Project` node under an `Aggregate`. These expressions are then referenced in both grouping expressions and aggregate expressions without aggregate functions to ensure that optimization rules don't change the aggregate expressions to invalid ones that no longer refer to any grouping expressions.

### Why are the changes needed?
If aggregate expressions (without aggregate functions) in an `Aggregate` node are complex then the `Optimizer` can optimize out grouping expressions from them and so making aggregate expressions invalid.

Here is a simple example:
```
SELECT not(t.id IS NULL) , count(*)
FROM t
GROUP BY t.id IS NULL
```
In this case the `BooleanSimplification` rule does this:
```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.BooleanSimplification ===
!Aggregate [isnull(id#222)], [NOT isnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]   Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]
 +- Project [value#219 AS id#222]                                                                 +- Project [value#219 AS id#222]
    +- LocalRelation [value#219]                                                                     +- LocalRelation [value#219]
```
where `NOT isnull(id#222)` is optimized to `isnotnull(id#222)` and so it no longer refers to any grouping expression.

Before this PR:
```
== Optimized Logical Plan ==
Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#234, count(1) AS c#232L]
+- Project [value#219 AS id#222]
   +- LocalRelation [value#219]
```
and running the query throws an error:
```
Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
java.lang.IllegalStateException: Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
```

After this PR:
```
== Optimized Logical Plan ==
Aggregate [_groupingexpression#233], [NOT _groupingexpression#233 AS (NOT (id IS NULL))#230, count(1) AS c#228L]
+- Project [isnull(value#219) AS _groupingexpression#233]
   +- LocalRelation [value#219]
```
and the query works.

### Does this PR introduce _any_ user-facing change?
Yes, the query works.

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

Closes #32396 from peter-toth/SPARK-34581-keep-grouping-expressions-2.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-02 05:53:09 +00:00
ulysses-you 39889df32a [SPARK-35264][SQL] Support AQE side broadcastJoin threshold
### What changes were proposed in this pull request?

~~This PR aims to add a new AQE optimizer rule `DynamicJoinSelection`. Like other AQE partition number configs, this rule add a new broadcast threshold config `spark.sql.adaptive.autoBroadcastJoinThreshold`.~~
This PR amis to add a flag in `Statistics` to distinguish AQE stats or normal stats, so that we can make some sql configs isolation between AQE and normal.

### Why are the changes needed?

The main idea here is that make join config isolation between normal planner and aqe planner which shared the same code path.

Actually we do not very trust using the static stats to consider if it can build broadcast hash join. In our experience it's very common that Spark throw broadcast timeout or driver side OOM exception when execute a bit large plan. And due to braodcast join is not reversed which means if we covert join to braodcast hash join at first time, we(AQE) can not optimize it again, so it should make sense to decide if we can do broadcast at aqe side using different sql config.

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

Yes, a new config `spark.sql.adaptive.autoBroadcastJoinThreshold` added.

### How was this patch tested?

Add new test.

Closes #32391 from ulysses-you/SPARK-35264.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-30 09:16:21 +00:00
Kousuke Saruta e8bf8fe213 [SPARK-35047][SQL] Allow Json datasources to write non-ascii characters as codepoints
### What changes were proposed in this pull request?

This PR proposes to enable the JSON datasources to write non-ascii characters as codepoints.
To enable/disable this feature, I introduce a new option `writeNonAsciiCharacterAsCodePoint` for JSON datasources.

### Why are the changes needed?

JSON specification allows codepoints as literal but Spark SQL's JSON datasources don't support the way to do it.
It's great if we can write non-ascii characters as codepoints, which is a platform neutral representation.

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

Yes. Users can write non-ascii characters as codepoints with JSON datasources.

### How was this patch tested?

New test.

Closes #32147 from sarutak/json-unicode-write.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-29 09:50:15 -07:00
Kousuke Saruta 132cbf0c8c [SPARK-35105][SQL] Support multiple paths for ADD FILE/JAR/ARCHIVE commands
### What changes were proposed in this pull request?

This PR extends `ADD FILE/JAR/ARCHIVE` commands to be able to take multiple path arguments like Hive.

### Why are the changes needed?

To make those commands more useful.

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

Yes. In the current implementation, those commands can take a path which contains whitespaces without enclose it by neither `'` nor `"` but after this change, users need to enclose such paths.
I've note this incompatibility in the migration guide.

### How was this patch tested?

New tests.

Closes #32205 from sarutak/add-multiple-files.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-29 13:58:51 +09:00
Kousuke Saruta 529b875901 [SPARK-35226][SQL] Support refreshKrb5Config option in JDBC datasources
### What changes were proposed in this pull request?

This PR proposes to introduce a new JDBC option `refreshKrb5Config` which allows to reflect the change of `krb5.conf`.

### Why are the changes needed?

In the current master, JDBC datasources can't accept `refreshKrb5Config` which is defined in `Krb5LoginModule`.
So even if we change the `krb5.conf` after establishing a connection, the change will not be reflected.

The similar issue happens when we run multiple `*KrbIntegrationSuites` at the same time.
`MiniKDC` starts and stops every KerberosIntegrationSuite and different port number is recorded to `krb5.conf`.
Due to `SecureConnectionProvider.JDBCConfiguration` doesn't take `refreshKrb5Config`, KerberosIntegrationSuites except the first running one see the wrong port so those suites fail.
You can easily confirm with the following command.
```
build/sbt -Phive Phive-thriftserver -Pdocker-integration-tests "testOnly org.apache.spark.sql.jdbc.*KrbIntegrationSuite"
```
### Does this PR introduce _any_ user-facing change?

Yes. Users can set `refreshKrb5Config` to refresh krb5 relevant configuration.

### How was this patch tested?

New test.

Closes #32344 from sarutak/kerberos-refresh-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-29 13:55:53 +09:00
Kent Yao 771356555c [SPARK-34786][SQL][FOLLOWUP] Explicitly declare DecimalType(20, 0) for Parquet UINT_64
### What changes were proposed in this pull request?

Explicitly declare DecimalType(20, 0) for Parquet UINT_64, avoid use DecimalType.LongDecimal which only happens to have 20 as precision.

https://github.com/apache/spark/pull/31960#discussion_r622691560

### Why are the changes needed?

fix ambiguity

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

no

### How was this patch tested?

not needed, just current CI pass

Closes #32390 from yaooqinn/SPARK-34786-F.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-29 04:51:27 +00:00
Chao Sun 86d3bb5f7d [SPARK-34981][SQL] Implement V2 function resolution and evaluation
Co-Authored-By: Chao Sun <sunchaoapple.com>
Co-Authored-By: Ryan Blue <rbluenetflix.com>

### What changes were proposed in this pull request?

This implements function resolution and evaluation for functions registered through V2 FunctionCatalog [SPARK-27658](https://issues.apache.org/jira/browse/SPARK-27658). In particular:
- Added documentation for how to define the "magic method" in `ScalarFunction`.
- Added a new expression `ApplyFunctionExpression` which evaluates input by delegating to `ScalarFunction.produceResult` method.
- added a new expression `V2Aggregator` which is a type of `TypedImperativeAggregate`. It's a wrapper of V2 `AggregateFunction` and mostly delegate methods to the implementation of the latter. It also uses plain Java serde for intermediate state.
- Added function resolution logic for `ScalarFunction` and `AggregateFunction` in `Analyzer`.
  + For `ScalarFunction` this checks if the magic method is implemented through Java reflection, and create a `Invoke` expression if so. Otherwise, it checks if the default `produceResult` is overridden. If so, it creates a `ApplyFunctionExpression` which evaluates through `InternalRow`. Otherwise an analysis exception is thrown.
 + For `AggregateFunction`, this checks if the `update` method is overridden. If so, it converts it to `V2Aggregator`. Otherwise an analysis exception is thrown similar to the case of `ScalarFunction`.
- Extended existing `InMemoryTableCatalog` to add the function catalog capability. Also renamed it to `InMemoryCatalog` since it no longer only covers tables.

**Note**: this currently can successfully detect whether a subclass overrides the default `produceResult` or `update` method from the parent interface **only for Java implementations**. It seems in Scala it's hard to differentiate whether a subclass overrides a default method from its parent interface. In this case, it will be a runtime error instead of analysis error.

A few TODOs:
- Extend `V2SessionCatalog` with function catalog. This seems a little tricky since API such V2 `FunctionCatalog`'s `loadFunction` is different from V1 `SessionCatalog`'s `lookupFunction`.
- Add magic method for `AggregateFunction`.
- Type coercion when looking up functions

### Why are the changes needed?

As V2 FunctionCatalog APIs are finalized, we should integrate it with function resolution and evaluation process so that they are actually useful.

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

Yes, now a function exposed through V2 FunctionCatalog can be analyzed and evaluated.

### How was this patch tested?

Added new unit tests.

Closes #32082 from sunchao/resolve-func-v2.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Chao Sun <sunchao@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-28 17:21:49 +00:00
ulysses-you 0bcf348438 [SPARK-34781][SQL][FOLLOWUP] Adjust the order of AQE optimizer rules
### What changes were proposed in this pull request?

Reorder  `DemoteBroadcastHashJoin` and `EliminateUnnecessaryJoin`.

### Why are the changes needed?

Skip unnecessary check in `DemoteBroadcastHashJoin` if `EliminateUnnecessaryJoin` affects.

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

No

### How was this patch tested?

No result affect.

Closes #32380 from ulysses-you/SPARK-34781-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-28 13:59:24 +00:00
ulysses-you 8b62c2964d [SPARK-35214][SQL] OptimizeSkewedJoin support ShuffledHashJoinExec
### What changes were proposed in this pull request?

Add `ShuffledHashJoin` pattern check in `OptimizeSkewedJoin` so that we can optimize it.

### Why are the changes needed?

Currently, we have already supported all type of join through hint that make it easy to choose the join implementation.

We would choose `ShuffledHashJoin` if one table is not big but over the broadcast threshold. It's better that we can support optimize it in `OptimizeSkewedJoin`.

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

Probably yes, the execute plan in AQE mode may be changed.

### How was this patch tested?

Improve exists test in `AdaptiveQueryExecSuite`

Closes #32328 from ulysses-you/SPARK-35214.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-28 16:57:57 +09:00
PengLei 046c8c3dd6 [SPARK-34878][SQL][TESTS] Check actual sizes of year-month and day-time intervals
### What changes were proposed in this pull request?
As we have suport the year-month and day-time intervals.  Add the test actual size of year-month and day-time intervals type

### Why are the changes needed?
Just add test

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

### How was this patch tested?
./dev/scalastyle
run test for "ColumnTypeSuite"

Closes #32366 from Peng-Lei/SPARK-34878.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-28 07:48:49 +03:00
Kousuke Saruta abb1f0c5d7 [SPARK-35236][SQL] Support archive files as resources for CREATE FUNCTION USING syntax
### What changes were proposed in this pull request?

This PR proposes to make `CREATE FUNCTION USING` syntax can take archives as resources.

### Why are the changes needed?

It would be useful.
`CREATE FUNCTION USING` syntax doesn't support archives as resources because archives were not supported in Spark SQL.
Now Spark SQL supports archives so I think we can support them for the syntax.

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

Yes. Users can specify archives for `CREATE FUNCTION USING` syntax.

### How was this patch tested?

New test.

Closes #32359 from sarutak/load-function-using-archive.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-28 10:15:21 +09:00
Kent Yao 16d223efee [SPARK-35091][SPARK-35090][SQL] Support extract from ANSI Intervals
### What changes were proposed in this pull request?

In this PR, we add extract/date_part support for ANSI Intervals

The `extract` is an ANSI expression and `date_part` is NON-ANSI but exists as an equivalence for `extract`

#### expression

```
<extract expression> ::=
  EXTRACT <left paren> <extract field> FROM <extract source> <right paren>
```

#### <extract field> for interval source

```

<primary datetime field> ::=
    <non-second primary datetime field>
| SECOND
<non-second primary datetime field> ::=
    YEAR
  | MONTH
  | DAY
  | HOUR
  | MINUTE
```

#### dataType

```
If <extract field> is a <primary datetime field> that does not specify SECOND or <extract field> is not a <primary datetime field>, then the declared type of the result is an implementation-defined exact numeric type with scale 0 (zero)

Otherwise, the declared type of the result is an implementation-defined exact numeric type with scale not less than the specified or implied <time fractional seconds precision> or <interval fractional seconds precision>, as appropriate, of the SECOND <primary datetime field> of the <extract source>.
```

### Why are the changes needed?

Subtask of ANSI Intervals Support

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

Yes
1. extract/date_part support ANSI intervals
2. for non-ansi intervals, the return type is changed from long to byte when extracting hours

### How was this patch tested?

new added tests

Closes #32351 from yaooqinn/SPARK-35091.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 13:06:54 +00:00
ulysses-you 4ff9f1fe3b [SPARK-35239][SQL] Coalesce shuffle partition should handle empty input RDD
### What changes were proposed in this pull request?

Create empty partition for custom shuffle reader if input RDD is empty.

### Why are the changes needed?

If input RDD partition is empty then the map output statistics will be null. And if all shuffle stage's input RDD partition is empty, we will skip it and lose the chance to coalesce partition.

We can simply create a empty partition for these custom shuffle reader to reduce the partition number.

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

Yes, the shuffle partition might be changed in AQE.

### How was this patch tested?

add new test.

Closes #32362 from ulysses-you/SPARK-35239.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 13:05:57 +00:00