Commit graph

11117 commits

Author SHA1 Message Date
Gengliang Wang 3951e3371a [SPARK-34881][SQL] New SQL Function: TRY_CAST
### What changes were proposed in this pull request?

Add a new SQL function `try_cast`.
`try_cast` is identical to  `AnsiCast` (or `Cast` when `spark.sql.ansi.enabled` is true), except it returns NULL instead of raising an error.
This expression has one major difference from `cast` with `spark.sql.ansi.enabled` as true: when the source value can't be stored in the target integral(Byte/Short/Int/Long) type, `try_cast` returns null instead of returning the low order bytes of the source value.
Note that the result of `try_cast` is not affected by the configuration `spark.sql.ansi.enabled`.

This is learned from Google BigQuery and Snowflake:
https://docs.snowflake.com/en/sql-reference/functions/try_cast.html
https://cloud.google.com/bigquery/docs/reference/standard-sql/functions-and-operators#safe_casting

### Why are the changes needed?

This is an useful for the following scenarios:
1. When ANSI mode is on, users can choose `try_cast` an alternative way to run SQL without errors for certain operations.
2. When ANSI mode is off, users can use `try_cast` to get a more reasonable result for casting a value to an integral type: when an overflow error happens, `try_cast` returns null while `cast` returns the low order bytes of the source value.

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

Yes, adding a new function `try_cast`

### How was this patch tested?

Unit tests.

Closes #31982 from gengliangwang/tryCast.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-31 20:47:04 +08:00
yi.wu f05b940749 [SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join
### What changes were proposed in this pull request?

This PR introduces a new analysis rule `DeduplicateRelations`, which deduplicates any duplicate relations in a plan first and then deduplicates conflicting attributes(which resued the `dedupRight` of `ResolveReferences`).

### Why are the changes needed?

`CostBasedJoinReorder` could fail when applying on self-join, e.g.,

```scala
// test in JoinReorderSuite
test("join reorder with self-join") {
  val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
      .select(nameToAttr("t1.v-1-10"))
      .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5")))

    // this can fail
    Optimize.execute(plan.analyze)
}
```
Besides, with the new rule `DeduplicateRelations`, we'd be able to enable some optimizations, e.g., LeftSemiAnti pushdown, redundant project removal, as reflects in updated unit tests.

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

### How was this patch tested?

Added and updated unit tests.

Closes #31470 from Ngone51/join-reorder.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 14:28:35 +08:00
Angerszhuuuu eecc43cb52 [SPARK-34568][SQL] When SparkContext's conf not enable hive, we should respect enableHiveSupport() when build SparkSession too
### What changes were proposed in this pull request?
When SparkContext is initialed, if we want to start SparkSession, when we call
`SparkSession.builder.enableHiveSupport().getOrCreate()`, the SparkSession we created won't have hive support since
we have't reset existed SC's conf's `spark.sql.catalogImplementation`.
In this PR we use sharedState.conf to decide whether we should enable Hive Support.

### Why are the changes needed?
We should respect `enableHiveSupport`

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

### How was this patch tested?
Added UT

Closes #31680 from AngersZhuuuu/SPARK-34568.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 05:59:24 +00:00
Max Gekk 162f0560e6 [SPARK-34896][SQL] Return day-time interval from dates subtraction
### What changes were proposed in this pull request?
1. Add the SQL config `spark.sql.legacy.interval.enabled` which will control when Spark SQL should use `CalendarIntervalType` instead of ANSI intervals.
2. Modify the `SubtractDates` expression to return values of `DayTimeIntervalType` when `spark.sql.legacy.interval.enabled` is set to `false` (which is the default).

### Why are the changes needed?
To conform to the ANSI SQL standard which requires ANSI intervals as the result of dates subtraction, see
<img width="656" alt="Screenshot 2021-03-29 at 19 09 34" src="https://user-images.githubusercontent.com/1580697/112866455-7e2f0d00-90c2-11eb-96e6-3feb7eea7e09.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```
and some tests from `SQLQueryTestSuite`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #31996 from MaxGekk/subtract-dates-to-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-31 08:20:13 +03:00
Tim Armstrong 13b255fefd [SPARK-34909][SQL] Fix conversion of negative to unsigned in conv()
### What changes were proposed in this pull request?
Use `java.lang.Long.divideUnsigned()` to do integer division in `NumberConverter` to avoid a bug in `unsignedLongDiv` that produced invalid results.

### Why are the changes needed?
The previous results are incorrect, the result of the below query should be 45012021522523134134555
```
scala> spark.sql("select conv('-10', 11, 7)").show(20, 150)
+-----------------------+
|       conv(-10, 11, 7)|
+-----------------------+
|4501202152252313413456|
+-----------------------+
scala> spark.sql("select hex(conv('-10', 11, 7))").show(20, 150)
+----------------------------------------------+
|                         hex(conv(-10, 11, 7))|
+----------------------------------------------+
|3435303132303231353232353233313334313334353600|
+----------------------------------------------+
```

### Does this PR introduce _any_ user-facing change?
`conv()` will produce different results because the bug is fixed.

### How was this patch tested?
Added a simple unit test.

Closes #32006 from timarmstrong/conv-unsigned.

Authored-by: Tim Armstrong <tim.armstrong@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 12:58:29 +08:00
Takeshi Yamamuro 46f96e9ce1 [SPARK-34795][SQL][TESTS] Adds a new job in GitHub Actions to check the output of TPC-DS queries
### What changes were proposed in this pull request?

This PR proposes to add a new job in GitHub Actions to check the output of TPC-DS queries.

NOTE: I've checked that the new job took 17m 35s in the GitHub Actions env.

### Why are the changes needed?

There are some cases where we noticed runtime-realted bugs after merging commits (e.g. .SPARK-33822). Therefore, I think it is worth adding a new job in GitHub Actions to check query output of TPC-DS (sf=1).

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

No.

### How was this patch tested?

The new test added.

Closes #31886 from maropu/TPCDSQueryTestSuite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-31 09:48:19 +09:00
Gengliang Wang c902f77b42 [SPARK-34856][FOLLOWUP][SQL] Remove dead code from AnsiCast.typeCheckFailureMessage
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/31954/, Array type is allowed to be cast as String type. So the customized conversion failure message branch from AnsiCast.typeCheckFailureMessage won't be reached anymore.
This PR is to remove the dead code.

### Why are the changes needed?

Code clean up.

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

No

### How was this patch tested?

Just removing dead code.

Closes #32004 from gengliangwang/SPARK-34856-followup.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-30 14:22:43 -05:00
Ali Afroozeh bd0990e3e8 [SPARK-34906] Refactor TreeNode's children handling methods into specialized traits
### What changes were proposed in this pull request?
Spark query plan node hierarchy has specialized traits (or abstract classes) for handling nodes with fixed number of children, for example `UnaryExpression`, `UnaryNode` and `UnaryExec` for representing an expression, a logical plan and a physical plan with only one child, respectively. This PR refactors the `TreeNode` hierarchy by extracting the children handling functionality into the following traits. `UnaryExpression` and other similar classes now extend the corresponding new trait:
```
trait LeafLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  override final def children: Seq[T] = Nil
}

trait UnaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def child: T
  transient override final lazy val children: Seq[T] = child :: Nil
}

trait BinaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def left: T
  def right: T
  transient override final lazy val children: Seq[T] = left :: right :: Nil
}

trait TernaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def first: T
  def second: T
  def third: T
  transient override final lazy val children: Seq[T] = first :: second :: third :: Nil
}
```

This refactoring, which is part of a bigger effort to make tree transformations in Spark more efficient, has two benefits:
- It moves the children handling methods to a single place, instead of being spread in specific subclasses, which will help the future optimizations for tree traversals.
- It allows to mix in these traits with some concrete node types that could not extend the previous classes. For example, expressions with one child that extend `AggregateFunction` cannot extend `UnaryExpression` as `AggregateFunction` defines the `foldable` method final while `UnaryExpression` defines it as non final. With the new traits, we can directly extend the concrete class from `UnaryLike` in these cases. Classes with more specific child handling will make tree traversal methods faster.

In this PR we have also updated many concrete node types to extend these traits to benefit from more specific child handling.

### Why are the changes needed?

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

### How was this patch tested?

This is a refactoring, passes existing tests.

Closes #31932 from dbaliafroozeh/FactorOutChildHandlnigIntoSeparateTraits.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-03-30 20:43:18 +02:00
ulysses-you 24d39a5ee2 [SPARK-34899][SQL] Use origin plan if we can not coalesce shuffle partition
### What changes were proposed in this pull request?

Add check if `CoalesceShufflePartitions` really coalesce shuffle partition number.

### Why are the changes needed?

The `CoalesceShufflePartitions` can not coalesce such case if the total shuffle partitions size of mappers are big enough. Then it's confused to use `CustomShuffleReaderExec` which marked as `coalesced` but has no affect with partition number.

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

Probably yes, the plan changed.

### How was this patch tested?

Add test.

Closes #31994 from ulysses-you/SPARK-34899.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 13:50:19 +00:00
Yuming Wang de66fa63f9 [SPARK-34884][SQL] Improve DPP evaluation to make filtering side must can broadcast by size or broadcast by hint
### What changes were proposed in this pull request?

Improve dynamic partition pruning evaluation to make filtering side must can broadcast by size or broadcast by hint.

### Why are the changes needed?

1. Fast fail if filtering side can not broadcast by size or broadcast by hint.
2. We can safely disable `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly`.

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

No.

### How was this patch tested?

Existing unit test.

Closes #31984 from wangyum/SPARK-34884.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 12:34:46 +00:00
angerszhu a98dc60408 [SPARK-33308][SQL] Refactor current grouping analytics
### What changes were proposed in this pull request?
As discussed in
https://github.com/apache/spark/pull/30145#discussion_r514728642
https://github.com/apache/spark/pull/30145#discussion_r514734648

We need to rewrite current Grouping Analytics grammar to support  as flexible as Postgres SQL to support subsequent development.
In  postgres sql, it support
```
select a, b, c, count(1) from t group by cube (a, b, c);
select a, b, c, count(1) from t group by cube(a, b, c);
select a, b, c, count(1) from t group by cube (a, b, c, (a, b), (a, b, c));
select a, b, c, count(1) from t group by rollup(a, b, c);
select a, b, c, count(1) from t group by rollup (a, b, c);
select a, b, c, count(1) from t group by rollup (a, b, c, (a, b), (a, b, c));
```
In this pr,  we have done three things as below, and we will split it to different pr:

 - Refactor CUBE/ROLLUP (regarding them as ANTLR tokens in a parser)
 - Refactor GROUPING SETS (the logical node -> a new expr)
 - Support new syntax for CUBE/ROLLUP (e.g., GROUP BY CUBE ((a, b), (a, c)))

### Why are the changes needed?
Rewrite current Grouping Analytics grammar to support  as flexible as Postgres SQL to support subsequent development.

### Does this PR introduce _any_ user-facing change?
User can  write Grouping Analytics grammar as flexible as Postgres SQL to support subsequent development.

### How was this patch tested?
Added UT

Closes #30212 from AngersZhuuuu/refact-grouping-analytics.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 12:31:58 +00:00
Cheng Su 935aa8c8db [SPARK-32985][SQL][FOLLOWUP] Rename createNonBucketedReadRDD and minor change in FileSourceScanExec
### What changes were proposed in this pull request?

This PR is a followup change to address comments in https://github.com/apache/spark/pull/31413#discussion_r603280965 and https://github.com/apache/spark/pull/31413#discussion_r603296475 . Minor change in `FileSourceScanExec`. No actual logic change here.

### Why are the changes needed?

Better readability.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #32000 from c21/bucket-scan.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 19:57:32 +09:00
David Li 1237124062 [SPARK-34463][PYSPARK][DOCS] Document caveats of Arrow selfDestruct
### What changes were proposed in this pull request?

As a followup for #29818, document caveats of using the Arrow selfDestruct option in toPandas, which include:
- toPandas() may be slower;
- the resulting dataframe may not support some Pandas operations due to immutable backing arrays.

### Why are the changes needed?

This will hopefully reduce user confusion as with SPARK-34463.

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

Yes - documentation is updated and a config setting description is updated to clearly indicate the config is experimental.

### How was this patch tested?
This is a documentation-only change.

Closes #31738 from lidavidm/spark-34463.

Authored-by: David Li <li.davidm96@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 13:30:27 +09:00
yangjie01 7158e7f986 [SPARK-34900][TEST] Make sure benchmarks can run using spark-submit cmd described in the guide
### What changes were proposed in this pull request?
Some `spark-submit`  commands used to run benchmarks in the user's guide is wrong, we can't use these commands to run benchmarks successful.

So the major changes of this pr is correct these wrong commands, for example, run a benchmark which inherits from `SqlBasedBenchmark`, we must specify `--jars <spark core test jar>,<spark catalyst test jar>` because `SqlBasedBenchmark` based benchmark extends `BenchmarkBase(defined in spark core test jar)` and `SQLHelper(defined in spark catalyst test jar)`.

Another change of this pr is removed the `scalatest Assertions` dependency of Benchmarks because `scalatest-*.jar` are not in the distribution package, it will be troublesome to use.

### Why are the changes needed?
Make sure benchmarks can run using spark-submit cmd described in the guide

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

### How was this patch tested?
Use the corrected `spark-submit` commands to run benchmarks successfully.

Closes #31995 from LuciferYang/fix-benchmark-guide.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 11:58:01 +09:00
Yuming Wang fcef2375a3 [SPARK-34622][SQL] Push down limit through Project with Join
### What changes were proposed in this pull request?

There is a `Project` between `LocalLimit` and `Join` if `Join`'s output do not match the `LocalLimit`'s output. This pr add support push down limit through this case. For example:
   ```scala
   spark.sql("create table t1(a int, b int, c int) using parquet")
   spark.sql("create table t2(x int, y int, z int) using parquet")
   spark.sql("select a from t1 left join t2 on a = x and b = y limit 5").explain("extended")
   ```

   ```
   == Optimized Logical Plan ==
   GlobalLimit 5
   +- LocalLimit 5
      +- Project [a#0]
         +- Join LeftOuter, ((a#0 = x#3) AND (b#1 = y#4))
            :- Project [a#0, b#1]
            :  +- Relation default.t1[a#0,b#1,c#2] parquet
            +- Project [x#3, y#4]
               +- Filter (isnotnull(x#3) AND isnotnull(y#4))
                  +- Relation default.t2[x#3,y#4,z#5] parquet
   ```

   After this pr:
   ```
   == Optimized Logical Plan ==
   GlobalLimit 5
   +- LocalLimit 5
      +- Project [a#0]
         +- Join LeftOuter, ((a#0 = x#3) AND (b#1 = y#4))
            :- LocalLimit 5
            :  +- Project [a#0, b#1]
            :     +- Relation default.t1[a#0,b#1,c#2] parquet
            +- Project [x#3, y#4]
               +- Filter (isnotnull(x#3) AND isnotnull(y#4))
                  +- Relation default.t2[x#3,y#4,z#5] parquet
   ```

### Why are the changes needed?

Improve limit push down to improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31739 from wangyum/SPARK-34622.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-30 10:45:30 +09:00
Jungtaek Lim 43e08b1f0f [SPARK-34255][SQL] Support partitioning with static number on required distribution and ordering on V2 write
### What changes were proposed in this pull request?

This PR proposes to extend the functionality of requirement for distribution and ordering on V2 write to specify the number of partitioning on repartition, so that data source is able to control the parallelism and determine the data distribution per partition in prior.

The partitioning with static number is optional, and by default disabled via default method, so only implementations required to restrict the number of partition statically need to override the method and provide the number.

Note that we don't support static number of partitions with unspecified distribution for this PR, as we haven't found the real use cases, and for hypothetical case the static number isn't good enough. Javadoc clearly describes the limitation.

### Why are the changes needed?

The use case comes from feature parity with DSv1.

I have state data source which enables the state in SS to be rewritten, which enables repartitioning, schema evolution, etc via batch query. The writer requires hash partitioning against group key, with the "desired number of partitions", which is same as what Spark does read and write against state.

This is now implemented as DSv1, and the requirement is simply done by calling repartition with the "desired number".

```
val fullPathsForKeyColumns = keySchema.map(key => new Column(s"key.${key.name}"))
data
  .repartition(newPartitions, fullPathsForKeyColumns: _*)
  .queryExecution
  .toRdd
  .foreachPartition(
    writeFn(resolvedCpLocation, version, operatorId, storeName, keySchema, valueSchema,
      storeConf, hadoopConfBroadcast, queryId))
```

Thanks to SPARK-34026, it's now possible to require the hash partitioning, but still not able to require the number of partitions. This PR will enable to let data source require the number of partitions.

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

Yes, but only for data source implementors. Even for them, this is no breaking change as default method is added.

### How was this patch tested?

Added UTs.

Closes #31355 from HeartSaVioR/SPARK-34255.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-29 14:33:23 +00:00
Kousuke Saruta 14c7bb877d [SPARK-34872][SQL] quoteIfNeeded should quote a name which contains non-word characters
### What changes were proposed in this pull request?

This PR fixes an issue that `quoteIfNeeded` quotes a name only if it contains `.` or ``` ` ```.
This method should quote it if it contains non-word characters.

### Why are the changes needed?

It's a potential bug.

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

No.

### How was this patch tested?

New test.

Closes #31964 from sarutak/fix-quoteIfNeeded.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-29 09:31:24 +00:00
Angerszhuuuu 015c59843c [SPARK-34879][SQL] HiveInspector supports DayTimeIntervalType and YearMonthIntervalType
### What changes were proposed in this pull request?
Make HiveInspector support DayTimeIntervalType and YearMonthIntervalType.
Then we can use these two types in HiveUDF and HiveScriptTransformation

### Why are the changes needed?
Support more data type when use hive serde

### Does this PR introduce _any_ user-facing change?
User can use  `DayTimeIntervalType` and `YearMonthIntervalType` in HiveUDF and  HiveScriptTransformation

### How was this patch tested?
Added UT

Closes #31979 from AngersZhuuuu/SPARK-34879.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-29 08:38:20 +03:00
Angerszhuuuu 2356cdd420 [SPARK-34814][SQL] LikeSimplification should handle NULL
### What changes were proposed in this pull request?
LikeSimplification should handle NULL.

UT will failed  before this pr
```
  test("SPARK-34814: LikeSimplification should handle NULL") {
    withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key ->
      ConstantFolding.getClass.getName.stripSuffix("$")) {
      checkEvaluation(Literal.create("foo", StringType)
        .likeAll("%foo%", Literal.create(null, StringType)), null)
    }
  }

[info] - test *** FAILED *** (2 seconds, 443 milliseconds)
[info]   java.lang.NullPointerException:
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$.$anonfun$simplifyMultiLike$1(expressions.scala:697)
[info]   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
[info]   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
[info]   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
[info]   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
[info]   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
[info]   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$.org$apache$spark$sql$catalyst$optimizer$LikeSimplification$$simplifyMultiLike(expressions.scala:697)
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$$anonfun$apply$9.applyOrElse(expressions.scala:722)
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$$anonfun$apply$9.applyOrElse(expressions.scala:714)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:316)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:316)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
[info]   at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$transformExpressionsDown$1(QueryPlan.scala:94)
[info]   at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:116)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
```

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #31976 from AngersZhuuuu/SPARK-34814.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-29 12:05:00 +09:00
Tanel Kiis 4b9e94c444 [SPARK-34876][SQL] Fill defaultResult of non-nullable aggregates
### What changes were proposed in this pull request?

Filled the `defaultResult` field on non-nullable aggregates

### Why are the changes needed?

The `defaultResult` defaults to `None` and in some situations (like correlated scalar subqueries) it is used for the value of the aggregation.

The UT result before the fix:
```
-- !query
SELECT t1a,
   (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2,
   (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2,
   (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2,
   (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2,
   (SELECT collect_set(t2d) FROM t2 WHERE t2a = t1a) collect_set_t2,
    (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2
FROM t1
-- !query schema
struct<t1a:string,count_t2:bigint,count_if_t2:bigint,approx_count_distinct_t2:bigint,collect_list_t2:array<bigint>,collect_set_t2:array<bigint>,collect_set_t2:string>
-- !query output
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1b	6	6	3	[19,119,319,19,19,19]	[19,119,319]	0000000100000000000000060000000100000004000000005D8D6AB90000000000000000000000000000000400000000000000010000000000000001
val1c	2	2	2	[219,19]	[219,19]	0000000100000000000000020000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000001
val1d	0	0	NULL	NULL	NULL	NULL
val1d	0	0	NULL	NULL	NULL	NULL
val1d	0	0	NULL	NULL	NULL	NULL
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
```

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

Bugfix

### How was this patch tested?

UT

Closes #31973 from tanelk/SPARK-34876_non_nullable_agg_subquery.

Authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-29 11:47:08 +09:00
hanover-fiste 4fceef0159 [SPARK-34843][SQL] Calculate more precise partition stride in JDBCRelation
### What changes were proposed in this pull request?
The changes being proposed are to increase the accuracy of JDBCRelation's stride calculation, as outlined in: https://issues.apache.org/jira/browse/SPARK-34843

In summary:

Currently, in JDBCRelation (line 123), the stride size is calculated as follows:
val stride: Long = upperBound / numPartitions - lowerBound / numPartitions

Due to truncation happening on both divisions, the stride size can fall short of what it should be. This can lead to a big difference between the provided upper bound and the actual start of the last partition.

I'm proposing a different formula that doesn't truncate to early, and also maintains accuracy using fixed-point decimals. This helps tremendously with the size of the last partition, which can be even more amplified if there is data skew in that direction. In a real-life test, I've seen a 27% increase in performance with this more proper stride alignment. The reason for fixed-point decimals instead of floating-point decimals is because inaccuracy due to limitation of what the float can represent. This may seem small, but could shift the midpoint a bit, and depending on how granular the data is, that could translate to quite a difference. It's also just inaccurate, and I'm striving to make the partitioning as accurate as possible, within reason.

Lastly, since the last partition's predicate is determined by how the strides align starting from the lower bound (plus one stride), there can be skew introduced creating a larger last partition compared to the first partition. Therefore, after calculating a more precise stride size, I've also introduced logic to move the first partition's predicate (which is an offset from the lower bound) to a position that closely matches the offset of the last partition's predicate (in relation to the upper bound). This makes the first and last partition more evenly distributed compared to each other, and helps with the last task being the largest (reducing its size).

### Why are the changes needed?
The current implementation is inaccurate and can lead to the last task/partition running much longer than previous tasks. Therefore, you can end up with a single node/core running for an extended period while other nodes/cores are sitting idle.

### Does this PR introduce _any_ user-facing change?
No. I would suspect some users will just get a good performance increase. As stated above, if we were to run our code on Spark that has this change implemented, we would have all of the sudden got a 27% increase in performance.

### How was this patch tested?
I've added two new unit tests. I did need to update one unit test, but when you look at the comparison of the before and after, you'll see better alignment of the partitioning with the new implementation. Given that the lower partition's predicate is exclusive and the upper's is inclusive, the offset of the lower was 3 days, and the offset of the upper was 6 days... that's potentially twice the amount of data in that upper partition (could be much more depending on how the user's data is distributed).

Other unit tests that utilize timestamps and two partitions have maintained their midpoint.

### Examples

I've added results with and without the realignment logic to better highlight both improvements this PR brings.

**Example 1:**
Given the following partition config:
"lowerBound" -> "1930-01-01"
"upperBound" -> "2020-12-31"
"numPartitions" -> 1000

_Old method (exactly what it would be BEFORE this PR):_
First partition: "PartitionColumn" < '1930-02-02' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2017-07-11'
_Old method, but with new realingment logic of first partition:_
First partition: "PartitionColumn" < '1931-10-14' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2019-03-22'

_New method:_
First partition: "PartitionColumn" < '1930-02-03' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-04-05'
_New with new realingment logic of first partition (exactly what it would be AFTER this PR):_
First partition: "PartitionColumn" < '1930-06-02' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-08-02'

**Example 2:**
Given the following partition config:
"lowerBound" -> "1927-04-05",
"upperBound" -> "2020-10-16"
"numPartitions" -> 2000

_Old method (exactly what it would be BEFORE this PR):_
First partition: "PartitionColumn" < '1927-04-21' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2014-10-29'
_Old method, but with new realingment logic of first partition::_
First partition: "PartitionColumn" < '1930-04-07' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2017-10-15'

_New method:_
First partition: "PartitionColumn" < '1927-04-22' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-04-19'
_New method with new realingment logic of first partition (exactly what it would be AFTER this PR):_
First partition: "PartitionColumn" < '1927-07-13' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-07-10'

Closes #31965 from hanover-fiste/SPARK-34843.

Authored-by: hanover-fiste <jyarbrough.git@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-28 12:59:20 -05:00
Peter Toth 3382190349 [SPARK-34829][SQL] Fix higher order function results
### What changes were proposed in this pull request?
This PR fixes a correctness issue with higher order functions. The results of function expressions needs to be copied in some higher order functions as such an expression can return with internal buffers and higher order functions can call multiple times the expression.
The issue was discovered with typed `ScalaUDF`s after https://github.com/apache/spark/pull/28979.

### Why are the changes needed?
To fix a bug.

### Does this PR introduce _any_ user-facing change?
Yes, some queries return the right results again.

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

Closes #31955 from peter-toth/SPARK-34829-fix-scalaudf-resultconversion.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 10:01:09 -07:00
Yuming Wang 540f1fb1d9 [SPARK-32855][SQL][FOLLOWUP] Fix code format in SQLConf and comment in PartitionPruning
### What changes were proposed in this pull request?

Fix code format in `SQLConf` and comment in `PartitionPruning`.

### Why are the changes needed?

Make code more readable.

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

No.

### How was this patch tested?

N/A

Closes #31969 from wangyum/SPARK-32855-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 09:48:54 -07:00
Dongjoon Hyun e7af44861e [SPARK-34880][SQL][TESTS] Add Parquet ZSTD compression test coverage
### What changes were proposed in this pull request?

Apache Parquet 1.12.0 switches its ZSTD compression from Hadoop codec to its own codec.

### Why are the changes needed?

**Apache Spark 3.1 (It requires libhadoop built with zstd)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
21/03/27 08:49:38 ERROR Executor: Exception in task 11.0 in stage 0.0 (TID 11)2]
java.lang.RuntimeException: native zStandard library not available:
this version of libhadoop was built without zstd support.
```

**Apache Spark 3.2 (No libhadoop requirement)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
```

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

Yes, this is an improvement.

### How was this patch tested?

Pass the CI with the newly added test coverage.

Closes #31981 from dongjoon-hyun/SPARK-34880.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 12:48:12 -07:00
Yuming Wang cbffc12f90 [SPARK-34542][BUILD] Upgrade Parquet to 1.12.0
### What changes were proposed in this pull request?

Parquet 1.12.0 New Feature
- PARQUET-41 - Add bloom filters to parquet statistics
- PARQUET-1373 - Encryption key management tools
- PARQUET-1396 - Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory
- PARQUET-1622 - Add BYTE_STREAM_SPLIT encoding
- PARQUET-1784 - Column-wise configuration
- PARQUET-1817 - Crypto Properties Factory
- PARQUET-1854 - Properties-Driven Interface to Parquet Encryption

Parquet 1.12.0 release notes:
https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/CHANGES.md

### Why are the changes needed?

- Bloom filters to improve filter performance
- ZSTD enhancement

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

No.

### How was this patch tested?

Existing unit test.

Closes #31649 from wangyum/SPARK-34542.

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-03-27 07:56:29 -07:00
Angerszhuuuu 468b944b00 [SPARK-34841][SQL] Push ANSI interval binary expressions into into (if/else) branches
### What changes were proposed in this pull request?
Push ANSI interval binary expressions into into (if / case) branches

### Why are the changes needed?
Support more binary expression to push into if/else and casewhen

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

### How was this patch tested?
Added UT

Closes #31978 from AngersZhuuuu/SPARK-34841.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-27 14:50:28 +03:00
Angerszhuuuu 769cf7b966 [SPARK-34744][SQL] Improve error message for casting cause overflow error
### What changes were proposed in this pull request?
Improve error message for casting cause overflow error. We should use DataType's catalogString.

### Why are the changes needed?
Improve error message

### Does this PR introduce _any_ user-facing change?
For example:
```
set spark.sql.ansi.enabled=true;
select tinyint(128) * tinyint(2);
```
Error message before this pr:
```
Casting 128 to scala.Byte$ causes overflow
```
After this pr:
```
Casting 128 to tinyint causes overflow
```

### How was this patch tested?
Added UT

Closes #31971 from AngersZhuuuu/SPARK-34744.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-27 11:15:55 +08:00
Max Gekk 9ba889b6ea [SPARK-34875][SQL] Support divide a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `day-time interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31972 from MaxGekk/div-dt-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 15:36:08 +00:00
Wenchen Fan 61d038f26e Revert "[SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand"
This reverts commit da04f1f4f8.
2021-03-26 15:26:48 +08:00
Max Gekk f212c61c43 [SPARK-34868][SQL] Support divide an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `year-month interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31961 from MaxGekk/div-ym-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 05:56:56 +00:00
Yuming Wang aaa0d2a66b [SPARK-32855][SQL] Improve the cost model in pruningHasBenefit for filtering side can not build broadcast by join type
### What changes were proposed in this pull request?

This pr improve the cost model in `pruningHasBenefit` for filtering side can not build broadcast by join type:
1. The filtering side must be small enough to build broadcast by size.
2. The estimated size of the pruning side must be big enough: `estimatePruningSideSize * spark.sql.optimizer.dynamicPartitionPruning.pruningSideExtraFilterRatio > overhead`.

### Why are the changes needed?

Improve query performance for these cases.

This a real case from cluster. Left join and left size very small and right side can build DPP:
![image](https://user-images.githubusercontent.com/5399861/92882197-445a2a00-f442-11ea-955d-16a7724e535b.png)

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

No.

### How was this patch tested?

Unit test.

Closes #29726 from wangyum/SPARK-32855.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 04:48:13 +00:00
Kent Yao 820b465886 [SPARK-34786][SQL] Read Parquet unsigned int64 logical type that stored as signed int64 physical type to decimal(20, 0)
### What changes were proposed in this pull request?

A companion PR for SPARK-34817, when we handle the unsigned int(<=32) logical types. In this PR, we map the unsigned int64 to decimal(20, 0) for better compatibility.

### Why are the changes needed?

Spark won't have unsigned types, but spark should be able to read existing parquet files written by other systems that support unsigned types for better compatibility.

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

yes, we can read parquet uint64 now

### How was this patch tested?

new unit tests

Closes #31960 from yaooqinn/SPARK-34786-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-26 09:54:19 +08:00
Yuanjian Li 5ffc3897e0 [SPARK-34871][SS] Move the checkpoint location resolving into the rule ResolveWriteToStream
### What changes were proposed in this pull request?
Move the checkpoint location resolving into the rule ResolveWriteToStream, which is added in SPARK-34748.

### Why are the changes needed?
After SPARK-34748, we have a rule ResolveWriteToStream for the analysis logic for the resolving logic of stream write plans. Based on it, we can further move the checkpoint location resolving work in the rule. Then, all the checkpoint resolving logic was done in the analyzer.

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

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

Closes #31963 from xuanyuanking/SPARK-34871.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-26 10:29:50 +09:00
Wenchen Fan 658e95c345 [SPARK-34833][SQL][FOLLOWUP] Handle outer references in all the places
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31940 . This PR generalizes the matching of attributes and outer references, so that outer references are handled everywhere.

Note that, currently correlated subquery has a lot of limitations in Spark, and the newly covered cases are not possible to happen. So this PR is a code refactor.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31959 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-26 09:10:03 +09:00
Gengliang Wang 0515f49018 [SPARK-34856][SQL] ANSI mode: Allow casting complex types as string type
### What changes were proposed in this pull request?

Allow casting complex types as string type in ANSI mode.

### Why are the changes needed?

Currently, complex types are not allowed to cast as string type. This breaks the DataFrame.show() API. E.g
```
scala> sql(“select array(1, 2, 2)“).show(false)
org.apache.spark.sql.AnalysisException: cannot resolve ‘CAST(`array(1, 2, 2)` AS STRING)’ due to data type mismatch:
 cannot cast array<int> to string with ANSI mode on.
```
We should allow the conversion as the extension of the ANSI SQL standard, so that the DataFrame.show() still work in ANSI mode.
### Does this PR introduce _any_ user-facing change?

Yes, casting complex types as string type is now allowed in ANSI mode.

### How was this patch tested?

Unit tests.

Closes #31954 from gengliangwang/fixExplicitCast.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-26 00:17:43 +08:00
Karen Feng 0d91f9c3f3 [SPARK-33600][SQL] Group exception messages in execution/datasources/v2
### What changes were proposed in this pull request?

This PR groups exception messages in `execution/datasources/v2`.

### 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 #31619 from karenfeng/spark-33600.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 16:15:30 +00:00
Tanel Kiis 6ba8445ea3 [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes
### What changes were proposed in this pull request?

Update the plan stability golden files even if only the `explain.txt` changes.

This is resubmition of #31927. The schema for one of the TPCDS tables was updated and that changed the `explain.txt` for the q17.

### Why are the changes needed?

Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`.

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

No

### How was this patch tested?

The updated golden files.

Closes #31957 from tanelk/SPARK-34822_update_plan_stability.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 10:22:49 +00:00
Tim Armstrong 1d6acd584a [SPARK-34857][SQL] Correct AtLeastNNonNulls's explain output
### What changes were proposed in this pull request?
Removed the custom toString implementation of AtLeastNNoneNulls.

### Why are the changes needed?
It shows up wrong in the explain plan. The name of the function is wrong and the actual value of the first argument is not shown. Both of these would make it easier to understand the plan.

```
(12) Filter
Input [3]: [c1#2410L, c2#2419, c3#2422]
Condition : AtLeastNNulls(n, c1#2410L)
```

### Does this PR introduce _any_ user-facing change?
Only the explain plan changes if this function is used.

### How was this patch tested?
Added a simple unit test to make sure that the toString output is correct.

Closes #31956 from timarmstrong/atleastnnonnulls.

Authored-by: Tim Armstrong <tim.armstrong@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-25 17:20:01 +09:00
Max Gekk a68d7ca8c5 [SPARK-34850][SQL] Support multiply a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * day-time interval` and `day-time interval * numeric`.
3. Invoke `DoubleMath.roundToInt` in `double/float * year-month interval`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31951 from MaxGekk/mul-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-25 10:46:50 +03:00
Kent Yao 8c6748f691 [SPARK-34817][SQL] Read parquet unsigned types that stored as int32 physical type in parquet
### What changes were proposed in this pull request?

Unsigned types may be used to produce smaller in-memory representations of the data. These types used by frameworks(e.g. hive, pig) using parquet. And parquet will map them to its base types.

see more https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift

```thrift
  /**
   * An unsigned integer value.
   *
   * The number describes the maximum number of meaningful data bits in
   * the stored value. 8, 16 and 32 bit values are stored using the
   * INT32 physical type.  64 bit values are stored using the INT64
   * physical type.
   *
   */
  UINT_8 = 11;
  UINT_16 = 12;
  UINT_32 = 13;
  UINT_64 = 14;
```

```
UInt8-[0:255]
UInt16-[0:65535]
UInt32-[0:4294967295]
UInt64-[0:18446744073709551615]
```

In this PR, we support read UINT_8 as ShortType, UINT_16 as IntegerType, UINT_32 as LongType to fit their range. Support for UINT_64 will be in another PR.

### Why are the changes needed?

better parquet support

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

yes, we can read unit[8/16/32] from parquet files

### How was this patch tested?

new tests

Closes #31921 from yaooqinn/SPARK-34817.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 06:58:06 +00:00
Terry Kim da04f1f4f8 [SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand
### What changes were proposed in this pull request?

This PR proposes to remove re-analyzing the already analyzed plan for `CreateViewCommand` as discussed https://github.com/apache/spark/pull/31273/files#r581592786.

### Why are the changes needed?

No need to analyze the plan if it's already analyzed.

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

No.

### How was this patch tested?

Existing tests should cover this.

Closes #31933 from imback82/remove_analyzed_from_create_temp_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 06:53:59 +00:00
HyukjinKwon 7838f55ca7 Revert "[SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes"
This reverts commit 84df54b495.
2021-03-25 12:31:08 +09:00
ulysses-you 9d561e6b5e [SPARK-34852][SQL] Close Hive session state should use withHiveState
### What changes were proposed in this pull request?

Wrap Hive sessionStae `close` with `withHiveState`

### Why are the changes needed?

Some reason:

1. Shutdown hook is invoked using different thread
2. Hive may use metasotre client again during closing

Otherwise, we may get such expcetion with custom hive metastore version
```
21/03/24 13:26:18 INFO session.SessionState: Failed to remove classloaders from DataNucleus
java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1654)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:80)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:130)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:101)
	at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3367)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3406)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3386)
	at org.apache.hadoop.hive.ql.session.SessionState.unCacheDataNucleusClassLoaders(SessionState.java:1546)
	at org.apache.hadoop.hive.ql.session.SessionState.close(SessionState.java:1536)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:172)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
```

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

No, since this not released.

### How was this patch tested?

manual test.

Closes #31949 from ulysses-you/SPARK-34852.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-25 10:21:44 +08:00
Takeshi Yamamuro 150769bced [SPARK-34833][SQL] Apply right-padding correctly for correlated subqueries
### What changes were proposed in this pull request?

This PR intends to fix the bug that does not apply right-padding for char types inside correlated subquries.
For example,  a query below returns nothing in master, but a correct result is `c`.
```
scala> sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING parquet")
scala> sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(7)) USING parquet")
scala> sql("INSERT INTO t1 VALUES ('c', 'b')")
scala> sql("INSERT INTO t2 VALUES ('a', 'b')")
scala> val df = sql("""
  |SELECT v FROM t1
  |WHERE 'a' IN (SELECT v FROM t2 WHERE t2.c = t1.c )""".stripMargin)

scala> df.show()
+---+
|  v|
+---+
+---+

```

This is because `ApplyCharTypePadding`  does not handle the case above to apply right-padding into `'abc'`. This PR modifies the code in `ApplyCharTypePadding` for handling it correctly.

```
// Before this PR:
scala> df.explain(true)
== Analyzed Logical Plan ==
v: string
Project [v#13]
+- Filter a IN (list#12 [c#14])
   :  +- Project [v#15]
   :     +- Filter (c#16 = outer(c#14))
   :        +- SubqueryAlias spark_catalog.default.t2
   :           +- Relation default.t2[v#15,c#16] parquet
   +- SubqueryAlias spark_catalog.default.t1
      +- Relation default.t1[v#13,c#14] parquet

scala> df.show()
+---+
|  v|
+---+
+---+

// After this PR:
scala> df.explain(true)
== Analyzed Logical Plan ==
v: string
Project [v#43]
+- Filter a IN (list#42 [c#44])
   :  +- Project [v#45]
   :     +- Filter (c#46 = rpad(outer(c#44), 7,  ))
   :        +- SubqueryAlias spark_catalog.default.t2
   :           +- Relation default.t2[v#45,c#46] parquet
   +- SubqueryAlias spark_catalog.default.t1
      +- Relation default.t1[v#43,c#44] parquet

scala> df.show()
+---+
|  v|
+---+
|  c|
+---+
```

This fix is lated to TPCDS q17; the query returns nothing because of this bug: https://github.com/apache/spark/pull/31886/files#r599333799

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Unit tests added.

Closes #31940 from maropu/FixCharPadding.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-25 08:31:57 +09:00
Gengliang Wang abfd9b23cd [SPARK-34769][SQL] AnsiTypeCoercion: return closest convertible type among TypeCollection
### What changes were proposed in this pull request?

Currently, when implicit casting a data type to a `TypeCollection`, Spark returns the first convertible data type among `TypeCollection`.
In ANSI mode, we can make the behavior more reasonable by returning the closet convertible data type in `TypeCollection`.

In details, we first try to find the all the expected types we can implicitly cast:
1. if there is no convertible data types, return None;
2. if there is only one convertible data type, cast input as it;
3. otherwise if there are multiple convertible data types, find the closet data
type among them. If there is no such closet data type, return None.

Note that if the closet type is Float type and the convertible types contains Double type, simply return Double type as the closet type to avoid potential
precision loss on converting the Integral type as Float type.

### Why are the changes needed?

Make the type coercion rule for TypeCollection more reasonable and ANSI compatible.
E.g. returning Long instead of Double for`implicast(int, TypeCollect(Double, Long))`.

From ANSI SQL Spec section 4.33 "SQL-invoked routines"
![Screen Shot 2021-03-17 at 4 05 06 PM](https://user-images.githubusercontent.com/1097932/111434916-5e104e80-86bd-11eb-8b3b-33090a68067d.png)

Section 9.6 "Subject routine determination"
![Screen Shot 2021-03-17 at 1 36 55 PM](https://user-images.githubusercontent.com/1097932/111420336-48445e80-86a8-11eb-9d50-34b325043bdb.png)

Section 10.4 "routine invocation"
![Screen Shot 2021-03-17 at 4 08 41 PM](https://user-images.githubusercontent.com/1097932/111434926-610b3f00-86bd-11eb-8c32-8c7935e055da.png)

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

Yes, in ANSI mode, implicit casting to a `TypeCollection` returns the narrowest convertible data type instead of the first convertible one.

### How was this patch tested?

Unit tests.

Closes #31859 from gengliangwang/implicitCastTypeCollection.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 15:04:03 +00:00
Tanel Kiis 84df54b495 [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes
### What changes were proposed in this pull request?

Update the plan stability golden files even if only the `explain.txt` changes.

### Why are the changes needed?

Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`.

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

No

### How was this patch tested?

The updated golden files.

Closes #31927 from tanelk/SPARK-34822_update_plan_stability.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 14:36:51 +00:00
Cheng Su 35c70e417d [SPARK-34853][SQL] Remove duplicated definition of output partitioning/ordering for limit operator
### What changes were proposed in this pull request?

Both local limit and global limit define the output partitioning and output ordering in the same way and this is duplicated (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L159-L175 ). We can move the output partitioning and ordering into their parent trait - `BaseLimitExec`. This is doable as `BaseLimitExec` has no more other child class. This is a minor code refactoring.

### Why are the changes needed?

Clean up the code a little bit. Better readability.

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

No.

### How was this patch tested?

Pure refactoring. Rely on existing unit tests.

Closes #31950 from c21/limit-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-24 23:06:35 +09:00
yangjie01 712a62ca82 [SPARK-34832][SQL][TEST] Set EXECUTOR_ALLOW_SPARK_CONTEXT to true to ensure ExternalAppendOnlyUnsafeRowArrayBenchmark run successfully
### What changes were proposed in this pull request?
SPARK-32160 add a config(`EXECUTOR_ALLOW_SPARK_CONTEXT`) to switch allow/disallow to create `SparkContext` in executors and the default value of the config is `false`

`ExternalAppendOnlyUnsafeRowArrayBenchmark` will run fail when `EXECUTOR_ALLOW_SPARK_CONTEXT` use the default value because the `ExternalAppendOnlyUnsafeRowArrayBenchmark#withFakeTaskContext` method try to create a `SparkContext` manually in Executor Side.

So the main change of this pr is  set `EXECUTOR_ALLOW_SPARK_CONTEXT` to `true` to ensure `ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

### Why are the changes needed?
Bug fix.

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

### How was this patch tested?
Manual test:
```
bin/spark-submit --class org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark --jars spark-core_2.12-3.2.0-SNAPSHOT-tests.jar spark-sql_2.12-3.2.0-SNAPSHOT-tests.jar
```

**Before**
```
Exception in thread "main" java.lang.IllegalStateException: SparkContext should only be created and accessed on the driver.
	at org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$assertOnDriver(SparkContext.scala:2679)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:89)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:137)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.withFakeTaskContext(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:52)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.testAgainstRawArrayBuffer(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:119)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.$anonfun$runBenchmarkSuite$1(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:189)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.benchmark.BenchmarkBase.runBenchmark(BenchmarkBase.scala:40)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.runBenchmarkSuite(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:186)
	at org.apache.spark.benchmark.BenchmarkBase.main(BenchmarkBase.scala:58)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark.main(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

**After**

`ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

Closes #31939 from LuciferYang/SPARK-34832.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-24 14:59:31 +09:00
Kousuke Saruta f7e9b6efc7 [SPARK-34763][SQL] col(), $"<name>" and df("name") should handle quoted column names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `col()`, `$"<name>"` and `df("name")` don't handle quoted column names  like ``` `a``b.c` ```properly.

For example, if we have a following DataFrame.
```
val df1 = spark.sql("SELECT 'col1' AS `a``b.c`")
```

For the DataFrame, this query is successfully executed.
```
scala> df1.selectExpr("`a``b.c`").show
+-----+
|a`b.c|
+-----+
| col1|
+-----+
```

But the following query will fail because ``` df1("`a``b.c`") ``` throws an exception.
```
scala> df1.select(df1("`a``b.c`")).show
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `a``b.c`;
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:152)
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:162)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:121)
  at org.apache.spark.sql.Dataset.resolve(Dataset.scala:221)
  at org.apache.spark.sql.Dataset.col(Dataset.scala:1274)
  at org.apache.spark.sql.Dataset.apply(Dataset.scala:1241)
  ... 49 elided
```
### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

New tests.

Closes #31854 from sarutak/fix-parseAttributeName.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 13:34:10 +08:00
Takeshi Yamamuro 0494dc90af [SPARK-34842][SQL][TESTS] Corrects the type of date_dim.d_quarter_name in the TPCDS schema
### What changes were proposed in this pull request?

SPARK-34842 (#31012) has a typo in the type of `date_dim.d_quarter_name` in the TPCDS schema (`TPCDSBase`). This PR replace `CHAR(1)` with `CHAR(6)`. This fix comes from p28 in [the TPCDS official doc](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf).

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

N/A

Closes #31943 from maropu/SPARK-34083-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-23 10:22:13 -07:00
Max Gekk 760556a42f [SPARK-34824][SQL] Support multiply an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * year-month interval` and `year-month interval * numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31929 from MaxGekk/interval-mul-div.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-23 19:40:15 +03:00
Wenchen Fan 3b70829b5b [SPARK-34719][SQL] Correctly resolve the view query with duplicated column names
forward-port https://github.com/apache/spark/pull/31811 to master

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

For permanent views (and the new SQL temp view in Spark 3.1), we store the view SQL text and re-parse/analyze the view SQL text when reading the view. In the case of `SELECT * FROM ...`, we want to avoid view schema change (e.g. the referenced table changes its schema) and will record the view query output column names when creating the view, so that when reading the view we can add a `SELECT recorded_column_names FROM ...` to retain the original view query schema.

In Spark 3.1 and before, the final SELECT is added after the analysis phase: https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala#L67

If the view query has duplicated output column names, we always pick the first column when reading a view. A simple repro:
```
scala> sql("create view c(x, y) as select 1 a, 2 a")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("select * from c").show
+---+---+
|  x|  y|
+---+---+
|  1|  1|
+---+---+
```

In the master branch, we will fail at the view reading time due to b891862fb6 , which adds the final SELECT during analysis, so that the query fails with `Reference 'a' is ambiguous`

This PR proposes to resolve the view query output column names from the matching attributes by ordinal.

For example,  `create view c(x, y) as select 1 a, 2 a`, the view query output column names are `[a, a]`. When we reading the view, there are 2 matching attributes (e.g.`[a#1, a#2]`) and we can simply match them by ordinal.

A negative example is
```
create table t(a int)
create view v as select *, 1 as col from t
replace table t(a int, col int)
```
When reading the view, the view query output column names are `[a, col]`, and there are two matching attributes of `col`, and we should fail the query. See the tests for details.

### Why are the changes needed?

bug fix

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

yes

### How was this patch tested?

new test

Closes #31930 from cloud-fan/view2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 14:34:51 +00:00
Liang-Chi Hsieh 115ed89a3c [SPARK-34366][SQL] Add interface for DS v2 metrics
### What changes were proposed in this pull request?

This patch proposes to add a few public API change to DS v2, to make DS v2 scan can report metrics to Spark.

Two public interfaces are added.

* `CustomMetric`: metric interface at the driver side. It basically defines how Spark aggregates task metrics with the same metric name.
* `CustomTaskMetric`: task metric reported at executors. It includes a name and long value. Spark will collect these metric values and update internal metrics.

There are two public methods added to existing public interfaces. They are optional to DS v2 implementations.

* `PartitionReader.currentMetricsValues()`: returns an array of CustomTaskMetric. Here is where the actual metrics values are collected. Empty array by default.
* `Scan.supportedCustomMetrics()`: returns an array of supported custom metrics `CustomMetric`. Empty array by default.

### Why are the changes needed?

In order to report custom metrics, we need some public API change in DS v2 to make it possible.

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

No

### How was this patch tested?

This only adds interfaces. In follow-up PRs where adding implementation there will be tests added. See #31451 and #31398 for some details and manual test there.

Closes #31476 from viirya/SPARK-34366.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 13:22:37 +00:00
Peter Toth 93a5d34f84 [SPARK-33482][SPARK-34756][SQL] Fix FileScan equality check
### What changes were proposed in this pull request?

This bug was introduced by SPARK-30428 at Apache Spark 3.0.0.
This PR fixes `FileScan.equals()`.

### Why are the changes needed?
- Without this fix `FileScan.equals` doesn't take `fileIndex` and `readSchema` into account.
- Partition filters and data filters added to `FileScan` (in #27112 and #27157) caused that canonicalized form of some `BatchScanExec` nodes don't match and this prevents some reuse possibilities.

### Does this PR introduce _any_ user-facing change?
Yes, before this fix incorrect reuse of `FileScan` and so `BatchScanExec` could have happed causing correctness issues.

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

Closes #31848 from peter-toth/SPARK-34756-fix-filescan-equality-check.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 17:01:16 +08:00
yi.wu e00afd31a7 [SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register inside itself
### What changes were proposed in this pull request?

Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into  itself.

Also with a minor change that put `registerSparkListenerForCleanup` to a better place.

### Why are the changes needed?

improve code

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

No.

### How was this patch tested?

Pass existing tests.

Closes #31919 from Ngone51/SPARK-34087-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:38:43 +00:00
linzebing e768eaa908 [SPARK-34707][SQL] Code-gen broadcast nested loop join (left outer/right outer)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left outer (build right) and right outer (build left). Reference: `BroadcastNestedLoopJoinExec.codegenInner()` and `BroadcastNestedLoopJoinExec.outerJoin()`

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:
```scala
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left outer broadcast nested loop join", N) {
   val df = spark.range(N).selectExpr(s"id as k1").join(
     dim, col("k1") + 1 <= col("k2"), "left_outer")
   assert(df.queryExecution.sparkPlan.find(
     _.isInstanceOf[BroadcastNestedLoopJoinExec]).isDefined)
   df.noop()
}
```
Seeing 2x run time improvement:
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left outer broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------------
left outer broadcast nested loop join wholestage off           3024           3698         953          6.9         144.2       1.0X
left outer broadcast nested loop join wholestage on            1512           1659         172         13.9          72.1       2.0X
```

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

No

### How was this patch tested?

Changed existing unit tests in `OuterJoinSuite` to cover codegen use cases.
Added unit test in WholeStageCodegenSuite.scala to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:
```scala
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_outer").explain("codegen")
```
Example generated code (`bnlj_doConsume_0` method):
```java
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, LeftOuter, ((k1#2L + 1) <= k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=16)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=16)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(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 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_foundMatch_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */       boolean bnlj_shouldOutputRow_0 = false;
/* 041 */
/* 042 */       boolean bnlj_isNull_2 = true;
/* 043 */       long bnlj_value_2 = -1L;
/* 044 */       if (bnlj_buildRow_0 != null) {
/* 045 */         long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 046 */         bnlj_isNull_2 = false;
/* 047 */         bnlj_value_2 = bnlj_value_1;
/* 048 */       }
/* 049 */
/* 050 */       long bnlj_value_4 = -1L;
/* 051 */
/* 052 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 053 */
/* 054 */       boolean bnlj_value_3 = false;
/* 055 */       bnlj_value_3 = bnlj_value_4 <= bnlj_value_2;
/* 056 */       if (!(false || !bnlj_value_3))
/* 057 */       {
/* 058 */         bnlj_shouldOutputRow_0 = true;
/* 059 */         bnlj_foundMatch_0 = true;
/* 060 */       }
/* 061 */       if (bnlj_arrayIndex_0 == bnlj_buildRowArray_0.length - 1 && !bnlj_foundMatch_0) {
/* 062 */         bnlj_buildRow_0 = null;
/* 063 */         bnlj_shouldOutputRow_0 = true;
/* 064 */       }
/* 065 */       if (bnlj_shouldOutputRow_0) {
/* 066 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 067 */
/* 068 */         boolean bnlj_isNull_9 = true;
/* 069 */         long bnlj_value_9 = -1L;
/* 070 */         if (bnlj_buildRow_0 != null) {
/* 071 */           long bnlj_value_8 = bnlj_buildRow_0.getLong(0);
/* 072 */           bnlj_isNull_9 = false;
/* 073 */           bnlj_value_9 = bnlj_value_8;
/* 074 */         }
/* 075 */         range_mutableStateArray_0[3].reset();
/* 076 */
/* 077 */         range_mutableStateArray_0[3].zeroOutNullBytes();
/* 078 */
/* 079 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 080 */
/* 081 */         if (bnlj_isNull_9) {
/* 082 */           range_mutableStateArray_0[3].setNullAt(1);
/* 083 */         } else {
/* 084 */           range_mutableStateArray_0[3].write(1, bnlj_value_9);
/* 085 */         }
/* 086 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 087 */
/* 088 */       }
/* 089 */     }
/* 090 */
/* 091 */   }
/* 092 */
/* 093 */   private void initRange(int idx) {
/* 094 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 095 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(16L);
/* 096 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 097 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 098 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 099 */     long partitionEnd;
/* 100 */
/* 101 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 102 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 103 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 104 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 105 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 106 */     } else {
/* 107 */       range_nextIndex_0 = st.longValue();
/* 108 */     }
/* 109 */     range_batchEnd_0 = range_nextIndex_0;
/* 110 */
/* 111 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 112 */     .multiply(step).add(start);
/* 113 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 114 */       partitionEnd = Long.MAX_VALUE;
/* 115 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 116 */       partitionEnd = Long.MIN_VALUE;
/* 117 */     } else {
/* 118 */       partitionEnd = end.longValue();
/* 119 */     }
/* 120 */
/* 121 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 122 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 123 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 124 */     if (range_numElementsTodo_0 < 0) {
/* 125 */       range_numElementsTodo_0 = 0;
/* 126 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 127 */       range_numElementsTodo_0++;
/* 128 */     }
/* 129 */   }
/* 130 */
/* 131 */   protected void processNext() throws java.io.IOException {
/* 132 */     // initialize Range
/* 133 */     if (!range_initRange_0) {
/* 134 */       range_initRange_0 = true;
/* 135 */       initRange(partitionIndex);
/* 136 */     }
/* 137 */
/* 138 */     while (true) {
/* 139 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 140 */         long range_nextBatchTodo_0;
/* 141 */         if (range_numElementsTodo_0 > 1000L) {
/* 142 */           range_nextBatchTodo_0 = 1000L;
/* 143 */           range_numElementsTodo_0 -= 1000L;
/* 144 */         } else {
/* 145 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 146 */           range_numElementsTodo_0 = 0;
/* 147 */           if (range_nextBatchTodo_0 == 0) break;
/* 148 */         }
/* 149 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 150 */       }
/* 151 */
/* 152 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 153 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 154 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 155 */
/* 156 */         // common sub-expressions
/* 157 */
/* 158 */         bnlj_doConsume_0(range_value_0);
/* 159 */
/* 160 */         if (shouldStop()) {
/* 161 */           range_nextIndex_0 = range_value_0 + 1L;
/* 162 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 163 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 164 */           return;
/* 165 */         }
/* 166 */
/* 167 */       }
/* 168 */       range_nextIndex_0 = range_batchEnd_0;
/* 169 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 170 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 171 */       range_taskContext_0.killTaskIfInterrupted();
/* 172 */     }
/* 173 */   }
/* 174 */
/* 175 */ }
```

Closes #31931 from linzebing/code-left-right-outer.

Authored-by: linzebing <linzebing1995@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:11:57 +00:00
hezuojiao 39542bb81f [SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled
### What changes were proposed in this pull request?

This patch proposes to disable fetching shuffle blocks in batch when io encryption is enabled. Adaptive Query Execution fetch contiguous shuffle blocks for the same map task in batch to reduce IO and improve performance. However, we found that batch fetching is incompatible with io encryption.

### Why are the changes needed?
Before this patch, we set `spark.io.encryption.enabled` to true, then run some queries which coalesced partitions by AEQ, may got following error message:
```14:05:52.638 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 2.0 (TID 3) (11.240.37.88 executor driver): FetchFailed(BlockManagerId(driver, 11.240.37.88, 63574, None), shuffleId=0, mapIndex=0, mapId=0, reduceId=2, message=
org.apache.spark.shuffle.FetchFailedException: Stream is corrupted
	at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:772)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:845)
	at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
	at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
	at java.io.DataInputStream.readInt(DataInputStream.java:387)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.readSize(UnsafeRowSerializer.scala:113)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:129)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:110)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:494)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:40)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	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:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
Caused by: java.io.IOException: Stream is corrupted
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:200)
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:226)
	at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:157)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:841)
	... 25 more

)
```

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

No

### How was this patch tested?

New tests.

Closes #31898 from hezuojiao/fetch_shuffle_in_batch.

Authored-by: hezuojiao <hezuojiao@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-22 13:06:12 -07:00
tanel.kiis@gmail.com 51cf0cadea [SPARK-34812][SQL] RowNumberLike and RankLike should not be nullable
### What changes were proposed in this pull request?

Marked `RowNumberLike` and `RankLike` as not-nullable.

### Why are the changes needed?

`RowNumberLike` and `RankLike` SQL expressions never return null value. Marking them as non-nullable can have some performance benefits, because some optimizer rules apply only to non-nullable expressions

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

No

### How was this patch tested?

Did not find any existing tests on the nullability of aggregate functions.
Plan stability suite partially covers this.

Closes #31924 from tanelk/SPARK-34812_nullability.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 14:55:43 +00:00
woyumen4597 f44608a8c0 [SPARK-34800][SQL] Use fine-grained lock in SessionCatalog.tableExists
### What changes were proposed in this pull request?
Use fine-grained lock in SessionCatalog.tableExists, in order to lock currentDB variable rather than lock `tableExists` method which will block inner external catalog's behaviour.

### Why are the changes needed?
We have modified the underlying hive meta store which a different hive  database is placed in its own shard for performance. However, we found that the synchronized lock  limits the concurrency.

### How was this patch tested?
Existing tests.

Closes #31891 from woyumen4597/SPARK-34800.

Authored-by: woyumen4597 <woyumen4597@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 09:03:46 +00:00
Terry Kim 7953fcdb56 [SPARK-34700][SQL] SessionCatalog's temporary view related APIs should take/return more concrete types
### What changes were proposed in this pull request?

Now that all the temporary views are wrapped with `TemporaryViewRelation`(#31273, #31652, and #31825), this PR proposes to update `SessionCatalog`'s APIs for temporary views to take or return more concrete types.

APIs that will take `TemporaryViewRelation` instead of `LogicalPlan`:
```
createTempView, createGlobalTempView, alterTempViewDefinition
```

APIs that will return `TemporaryViewRelation` instead of `LogicalPlan`:
```
getRawTempView, getRawGlobalTempView
```

APIs that will return `View` instead of `LogicalPlan`:
```
getTempView, getGlobalTempView, lookupTempView
```

### Why are the changes needed?

Internal refactoring to work with more concrete types.

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

No, this is internal refactoring.

### How was this patch tested?

Updated existing tests affected by the refactoring.

Closes #31906 from imback82/use_temporary_view_relation.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:17:54 +00:00
yi.wu e4bb97526c [SPARK-34089][CORE] HybridRowQueue should respect the configured memory mode
### What changes were proposed in this pull request?

This PR fixes the `HybridRowQueue ` to respect the configured memory mode.

Besides, this PR also refactored the constructor of `MemoryConsumer` to accept the memory mode explicitly.

### Why are the changes needed?

`HybridRowQueue` supports both onHeap and offHeap manipulation. But it inherited the wrong `MemoryConsumer` constructor, which hard-coded the memory mode to `onHeap`.

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

No. (Maybe yes in some cases where users can't complete the job before could complete successfully after the fix because of `HybridRowQueue` is able to spill under offHeap mode now. )

### How was this patch tested?

Updated the existing test to make it test both offHeap and onHeap modes.

Closes #31152 from Ngone51/fix-MemoryConsumer-memorymode.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:12:08 +00:00
HyukjinKwon ec70467d4d [SPARK-34815][SQL] Update CSVBenchmark
### What changes were proposed in this pull request?

This PR updates CSVBenchmark especially we have a fix like https://github.com/apache/spark/pull/31858 that could potentially improve the performance.

### Why are the changes needed?

To have the updated benchmark results.

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

No.

### How was this patch tested?

Manually ran the benchmark

Closes #31917 from HyukjinKwon/SPARK-34815.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 10:49:53 +03:00
Jungtaek Lim (HeartSaVioR) 121883b1a5 [SPARK-34383][SS] Optimize WAL commit phase via reducing cost of filesystem operations
### What changes were proposed in this pull request?

This PR proposes to optimize WAL commit phase via following changes:

* cache offset log to avoid FS get operation per batch
* just directly delete instead of employing FS list operation on purge

### Why are the changes needed?

There're inefficiency on WAL commit phase which can be easily optimized via using a small driver memory.

1. To provide the offset metadata to source side (via `source.commit()`), we read offset metadata for previous batch from file system, which is probably written by this driver in previous batches. Caching it into driver memory would reduce the get operation.
2. Spark calls purge against offset log & commit log per batch, which calls list operation. If the previous batch succeeded to purge, the current batch just needs to check one batch which can be simply done via direct delete operation, instead of calling list operation.

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

No.

### How was this patch tested?

Manually tested with additional debug log. (Verified that cache is used, cache keeps the size as 2, only one delete call is used instead of list call)

Did some experiment with simple rate to console query. (NOTE: wasn't done with master branch - tested against Spark 2.4.x, but WAL commit phase hasn't been changed AFAIK during these versions)

AWS S3 + S3 guard:

> before the patch

<img width="1075" alt="aws-before" src="https://user-images.githubusercontent.com/1317309/107108721-6cc54380-687d-11eb-8f10-b906b9d58397.png">

> after the patch

<img width="1071" alt="aws-after" src="https://user-images.githubusercontent.com/1317309/107108724-7189f780-687d-11eb-88da-26912ac15c85.png">

Azure:

> before the patch

<img width="1074" alt="azure-before" src="https://user-images.githubusercontent.com/1317309/107108726-75b61500-687d-11eb-8c06-9048fa10ff9a.png">

> after the patch

<img width="1069" alt="azure-after" src="https://user-images.githubusercontent.com/1317309/107108729-79e23280-687d-11eb-8d97-e7f3aeec51be.png">

Closes #31495 from HeartSaVioR/SPARK-34383.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
2021-03-22 08:47:07 +01:00
Cheng Su f8838fe82b [SPARK-34708][SQL] Code-gen for left semi/anti broadcast nested loop join (build right side)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left semi / left anti BroadcastNestedLoopJoin (build side is right side). The execution code path for build left side cannot fit into whole stage code-gen framework, so only add the code-gen for build right side here.

Reference: the iterator (non-code-gen) code path is `BroadcastNestedLoopJoinExec.leftExistenceJoin()` with `BuildRight`.

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:

```
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left semi broadcast nested loop join", N) {
  park.range(N).selectExpr(s"id as k1").join(
    dim, col("k1") + 1 <= col("k2"), "left_semi")
}
```

Seeing 5x run time improvement:

```
Running benchmark: left semi broadcast nested loop join
  Running case: left semi broadcast nested loop join codegen off
  Stopped after 2 iterations, 6958 ms
  Running case: left semi broadcast nested loop join codegen on
  Stopped after 5 iterations, 3383 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left semi broadcast nested loop join:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------------------------------------------------------
left semi broadcast nested loop join codegen off           3434           3479          65          6.1         163.7       1.0X
left semi broadcast nested loop join codegen on             672            677           5         31.2          32.1       5.1X
```

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

No.

### How was this patch tested?

Changed existing unit test in `ExistenceJoinSuite.scala` to cover all code paths:
* left semi/anti + empty right side + empty condition
* left semi/anti + non-empty right side + empty condition
* left semi/anti + right side + non-empty condition

Added unit test in `WholeStageCodegenSuite.scala` to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_semi").explain("codegen")
```

Example generated code (`bnlj_doConsume_0` method):
This is for left semi join. The generated code for left anti join is mostly to be same as here, except L55 to be `if (bnlj_findMatchedRow_0 == false) {`.
```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) Project [id#0L AS k1#2L]
+- *(2) BroadcastNestedLoopJoin BuildRight, LeftSemi, ((id#0L + 1) <= k2#6L)
   :- *(2) Range (0, 4, step=1, splits=2)
   +- BroadcastExchange IdentityBroadcastMode, [id=#23]
      +- *(1) Project [id#4L AS k2#6L]
         +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(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 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 031 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_findMatchedRow_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */
/* 041 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 042 */
/* 043 */       long bnlj_value_3 = -1L;
/* 044 */
/* 045 */       bnlj_value_3 = bnlj_expr_0_0 + 1L;
/* 046 */
/* 047 */       boolean bnlj_value_2 = false;
/* 048 */       bnlj_value_2 = bnlj_value_3 <= bnlj_value_1;
/* 049 */       if (!(false || !bnlj_value_2))
/* 050 */       {
/* 051 */         bnlj_findMatchedRow_0 = true;
/* 052 */         break;
/* 053 */       }
/* 054 */     }
/* 055 */     if (bnlj_findMatchedRow_0 == true) {
/* 056 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 057 */
/* 058 */       // common sub-expressions
/* 059 */
/* 060 */       range_mutableStateArray_0[3].reset();
/* 061 */
/* 062 */       range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 063 */       append((range_mutableStateArray_0[3].getRow()).copy());
/* 064 */
/* 065 */     }
/* 066 */
/* 067 */   }
/* 068 */
/* 069 */   private void initRange(int idx) {
/* 070 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 071 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 072 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 073 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 074 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 075 */     long partitionEnd;
/* 076 */
/* 077 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 078 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 079 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 080 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 081 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 082 */     } else {
/* 083 */       range_nextIndex_0 = st.longValue();
/* 084 */     }
/* 085 */     range_batchEnd_0 = range_nextIndex_0;
/* 086 */
/* 087 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 088 */     .multiply(step).add(start);
/* 089 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 090 */       partitionEnd = Long.MAX_VALUE;
/* 091 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 092 */       partitionEnd = Long.MIN_VALUE;
/* 093 */     } else {
/* 094 */       partitionEnd = end.longValue();
/* 095 */     }
/* 096 */
/* 097 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 098 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 099 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 100 */     if (range_numElementsTodo_0 < 0) {
/* 101 */       range_numElementsTodo_0 = 0;
/* 102 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 103 */       range_numElementsTodo_0++;
/* 104 */     }
/* 105 */   }
/* 106 */
/* 107 */   protected void processNext() throws java.io.IOException {
/* 108 */     // initialize Range
/* 109 */     if (!range_initRange_0) {
/* 110 */       range_initRange_0 = true;
/* 111 */       initRange(partitionIndex);
/* 112 */     }
/* 113 */
/* 114 */     while (true) {
/* 115 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 116 */         long range_nextBatchTodo_0;
/* 117 */         if (range_numElementsTodo_0 > 1000L) {
/* 118 */           range_nextBatchTodo_0 = 1000L;
/* 119 */           range_numElementsTodo_0 -= 1000L;
/* 120 */         } else {
/* 121 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 122 */           range_numElementsTodo_0 = 0;
/* 123 */           if (range_nextBatchTodo_0 == 0) break;
/* 124 */         }
/* 125 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 126 */       }
/* 127 */
/* 128 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 129 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 130 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 131 */
/* 132 */         bnlj_doConsume_0(range_value_0);
/* 133 */
/* 134 */         if (shouldStop()) {
/* 135 */           range_nextIndex_0 = range_value_0 + 1L;
/* 136 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 137 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 138 */           return;
/* 139 */         }
/* 140 */
/* 141 */       }
/* 142 */       range_nextIndex_0 = range_batchEnd_0;
/* 143 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 144 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 145 */       range_taskContext_0.killTaskIfInterrupted();
/* 146 */     }
/* 147 */   }
/* 148 */
/* 149 */ }
```

Closes #31874 from c21/code-semi-anti.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 07:31:16 +00:00
Yuanjian Li 45235ac4bc [SPARK-34748][SS] Create a rule of the analysis logic for streaming write
### What changes were proposed in this pull request?
- Create a new rule `ResolveStreamWrite` for all analysis logic for streaming write.
- Add corresponding logical plans `WriteToStreamStatement` and `WriteToStream`.

### Why are the changes needed?
Currently, the analysis logic for streaming write is mixed in StreamingQueryManager. If we create a specific analyzer rule and separated logical plans, it should be helpful for further extension.

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

### How was this patch tested?
Existing tests.

Closes #31842 from xuanyuanking/SPARK-34748.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 06:39:39 +00:00
Josh Soref f4de93efb0 [MINOR][SQL] Spelling: filters - PushedFilers
### What changes were proposed in this pull request?
Consistently correct the spelling of `PushedFilters`

### Why are the changes needed?
bersprockets noted that it's wrong

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

Technically, I think it does. Practically, neither Google nor GitHub show anyone using `pushedFilers` outside of forks (or the discussion about fixing it started at https://github.com/apache/spark/pull/30323#issuecomment-725568719)

### How was this patch tested?
None beyond CI in the previous PR

Closes #30678 from jsoref/spelling-filters.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 08:00:12 +03:00
Dongjoon Hyun c5fd94f119 [SPARK-34772][TESTS][FOLLOWUP] Disable a test case using Hive 1.2.1 in Java9+ environment
### What changes were proposed in this pull request?

This PR aims to disable a new test case using Hive 1.2.1 from Java9+ test environment.

### Why are the changes needed?

[HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113) upgraded Datanucleus to 4.x at Hive 2.0. Datanucleus 3.x doesn't support Java9+.

**Java 9+ Environment**
```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] *** 1 TEST FAILED ***
[error] Failed: Total 1, Failed 1, Errors 0, Passed 0
[error] Failed tests:
[error] 	org.apache.spark.sql.hive.HiveSparkSubmitSuite
[error] (hive / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful
[error] Total time: 328 s (05:28), completed Mar 21, 2021, 5:32:39 PM
```

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

Fix the UT in Java9+ environment.

### How was this patch tested?

Manually.

```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] HiveSparkSubmitSuite:
[info] - SPARK-34772: RebaseDateTime loadRebaseRecords should use Spark classloader instead of context !!! CANCELED !!! (26 milliseconds)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:344)
```

Closes #31916 from dongjoon-hyun/SPARK-HiveSparkSubmitSuite.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 17:59:55 -07:00
Kousuke Saruta 94fd6cb0ce [SPARK-34636][FOLLOWUP][SQL] Fix an incompatible behavior of UnresolvedAttribute.sql
### What changes were proposed in this pull request?

This PR fixes an incompatible behavior introduced by #31754.
The problem is that quoted name parts represented as a string are given to the constructor of `UnresolvedAttribute` which takes single string parameter, `sql` method invocation against the `UnresolvedAttrribute` returns different result than before.

One example is ``` UnresolvedAttribute("`a.b`").sql ```. This  returned `a.b` before but it doesn't now.

See [this duscussion](https://github.com/apache/spark/pull/31754/files#r597181927) for more details.

### Why are the changes needed?

For compatibility.

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

No.

### How was this patch tested?

New assertion.

Closes #31885 from sarutak/followup-SPARK-34636.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-20 14:44:36 -07:00
Yuming Wang 908318f30d [SPARK-28220][SQL] Improve PropagateEmptyRelation to support join with false condition
### What changes were proposed in this pull request?

Improve `PropagateEmptyRelation` to support join with false condition. For example:
```sql
SELECT * FROM t1 LEFT JOIN t2 ON false
```

Before this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- BroadcastNestedLoopJoin BuildRight, LeftOuter, false
   :- FileScan parquet default.t1[a#4L]
   +- BroadcastExchange IdentityBroadcastMode, [id=#40]
      +- FileScan parquet default.t2[b#5L]
```

After this pr:
```
== Physical Plan ==
*(1) Project [a#4L, null AS b#5L]
+- *(1) ColumnarToRow
   +- FileScan parquet default.t1[a#4L]
```

### Why are the changes needed?

Avoid `BroadcastNestedLoopJoin` to improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31857 from wangyum/SPARK-28220.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-20 22:57:02 +08:00
Kent Yao 2cdedef2a0 [SPARK-34128][SQL] Suppress undesirable TTransportException warnings involved in THRIFT-4805
### What changes were proposed in this pull request?

Since Spark 3.0, the `libthrift` has been bumped up from 0.9.3 to 0.12.0.

Due to THRIFT-4805, The SparkThrift Server will print annoying TExceptions. For example, the current thrift server module test in Github action workflow outputs more than 200MB of data for this error only
```java
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

I checked the latest `hive-service-rpc` module in the maven center,  https://mvnrepository.com/artifact/org.apache.hive/hive-service-rpc/3.1.2.  It still uses the 0.9.3 version.

Unfortunately, I tried the newly released `libthrift 0.14.1`(w/o shading it), it breaks the metastore client side.

```scala
java.lang.NoSuchMethodError: org.apache.thrift.transport.TSocket.<init>(Ljava/lang/String;II)V
```
On the Thrift side, they just muted it see https://issues.apache.org/jira/browse/THRIFT-4805

So in this PR, I add a filter to suppress the warning

### Why are the changes needed?

if the log is too large, the Github action might truncate it. We need to reduce useless output.

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

no

### How was this patch tested?

```build/sbt "hive-thriftserver/testOnly *ThriftServerQueryTestSuite" -Phive-thriftserver``` locally

#### before

```java
[info] - count.sql (1 second, 537 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
14:09:53.233 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - group-analytics.sql (4 seconds, 282 milliseconds)

[info] - csv-functions.sql (400 milliseconds)
14:09:24.234 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - datetime-formatting-invalid.sql (349 milliseconds)
14:09:26.544 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - except.sql (2 seconds, 309 milliseconds)
14:09:27.782 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - string-functions.sql (1 second, 237 milliseconds)
14:09:27.835 WARN org.apache.spark.sql.execution.datasources.DataSource: All paths were ignored:

14:09:29.266 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

```

#### after

```java

[info] - null-propagation.sql (181 milliseconds)
[info] - operators.sql (1 second, 772 milliseconds)
[info] - change-column.sql (241 milliseconds)
[info] - count.sql (1 second, 665 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
[info] - group-analytics.sql (3 seconds, 926 milliseconds)
[info] - inline-table.sql (247 milliseconds)
[info] - comparator.sql (223 milliseconds)
[info] - show-tblproperties.sql (148 milliseconds)
[info] - timezone.sql (105 milliseconds)
[info] - parse-schema-string.sql (193 milliseconds)
```

Closes #31895 from yaooqinn/SPARK-34128-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 21:15:28 -07:00
Cheng Su 2ff0032e01 [SPARK-34796][SQL] Initialize counter variable for LIMIT code-gen in doProduce()
### What changes were proposed in this pull request?

This PR is to fix the LIMIT code-gen bug in https://issues.apache.org/jira/browse/SPARK-34796, where the counter variable from `BaseLimitExec` is not initialized but used in code-gen. This is because the limit counter variable will be used in upstream operators (LIMIT's child plan, e.g. `ColumnarToRowExec` operator for early termination), but in the same stage, there can be some operators doing the shortcut and not calling `BaseLimitExec`'s `doConsume()`, e.g. [HashJoin.codegenInner](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L402). So if we have query that `LocalLimit - BroadcastHashJoin - FileScan` in the same stage, the whole stage code-gen compilation will be failed.

Here is an example:

```
  test("failed limit query") {
    withTable("left_table", "empty_right_table", "output_table") {
      spark.range(5).toDF("k").write.saveAsTable("left_table")
      spark.range(0).toDF("k").write.saveAsTable("empty_right_table")

      withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") {
        spark.sql("CREATE TABLE output_table (k INT) USING parquet")
        spark.sql(
          s"""
             |INSERT INTO TABLE output_table
             |SELECT t1.k FROM left_table t1
             |JOIN empty_right_table t2
             |ON t1.k = t2.k
             |LIMIT 3
             |""".stripMargin)
      }
    }
  }
```

Query plan:

```
Execute InsertIntoHadoopFsRelationCommand file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table, false, Parquet, Map(path -> file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table), Append, CatalogTable(
Database: default
Table: output_table
Created Time: Thu Mar 18 21:46:26 PDT 2021
Last Access: UNKNOWN
Created By: Spark 3.2.0-SNAPSHOT
Type: MANAGED
Provider: parquet
Location: file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table
Schema: root
 |-- k: integer (nullable = true)
), org.apache.spark.sql.execution.datasources.InMemoryFileIndexb25d08b, [k]
+- *(3) Project [ansi_cast(k#228L as int) AS k#231]
   +- *(3) GlobalLimit 3
      +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#179]
         +- *(2) LocalLimit 3
            +- *(2) Project [k#228L]
               +- *(2) BroadcastHashJoin [k#228L], [k#229L], Inner, BuildRight, false
                  :- *(2) Filter isnotnull(k#228L)
                  :  +- *(2) ColumnarToRow
                  :     +- FileScan parquet default.left_table[k#228L] Batched: true, DataFilters: [isnotnull(k#228L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
                  +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#173]
                     +- *(1) Filter isnotnull(k#229L)
                        +- *(1) ColumnarToRow
                           +- FileScan parquet default.empty_right_table[k#229L] Batched: true, DataFilters: [isnotnull(k#229L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
```

Codegen failure - https://gist.github.com/c21/ea760c75b546d903247582be656d9d66 .

The uninitialized variable `_limit_counter_1` from `LocalLimitExec` is referenced in `ColumnarToRowExec`, but `BroadcastHashJoinExec` does not call `LocalLimitExec.doConsume()` to initialize the counter variable.

The fix is to move the counter variable initialization to `doProduce()`, as in whole stage code-gen framework, `doProduce()` will definitely be called if upstream operators `doProduce()`/`doConsume()` is called.

Note: this only happens in AQE disabled case, because we have an AQE optimization rule [EliminateUnnecessaryJoin](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala#L69) to change the whole query to an empty `LocalRelation` if inner join broadcast side is empty with AQE enabled.

### Why are the changes needed?

Fix query failure.

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

No.

### How was this patch tested?

Added unit test in `SQLQuerySuite.scala`.

Closes #31892 from c21/limit-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:20:52 +09:00
tanel.kiis@gmail.com 620cae098c [SPARK-33122][SQL] Remove redundant aggregates in the Optimzier
### What changes were proposed in this pull request?

Added optimizer rule `RemoveRedundantAggregates`. It removes redundant aggregates from a query plan. A redundant aggregate is an aggregate whose only goal is to keep distinct values, while its parent aggregate would ignore duplicate values.

The affected part of the query plan for TPCDS q87:

Before:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#785]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
            +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                  +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                     +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#724]
                        +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                           +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                              :- ...
```

After:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#751]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#694]
            +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                  :- ...
```

### Why are the changes needed?

Performance improvements - few TPCDS queries have these kinds of duplicate aggregates.

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

No

### How was this patch tested?

UT

Benchmarks (sf=5):

OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Linux 5.8.13-arch1-1
Intel(R) Core(TM) i5-6500 CPU  3.20GHz

| Query | Before  | After | Speedup |
| ------| ------- | ------| ------- |
| q14a | 44s | 44s | 1x |
| q14b | 41s | 41s | 1x |
| q38  | 6.5s | 5.9s | 1.1x |
| q87  | 7.2s | 6.8s | 1.1x |
| q14a-v2.7 | 55s | 53s | 1x |

Closes #30018 from tanelk/SPARK-33122.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@reach-u.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:16:39 +09:00
Liang-Chi Hsieh 7a8a600995 [SPARK-34776][SQL] Nested column pruning should not prune Window produced attributes
### What changes were proposed in this pull request?

This patch proposes to fix a bug related to `NestedColumnAliasing`. The root cause is `Window`  doesn't override `producedAttributes` so `NestedColumnAliasing` rule wrongly prune attributes produced by `Window`.

The master and branch-3.1 both have this issue.

### Why are the changes needed?

It is needed to fix a bug of nested column pruning.

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

No

### How was this patch tested?

Unit test.

Closes #31897 from viirya/SPARK-34776.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 11:44:02 -07:00
Max Gekk 089c3b77e1 [SPARK-34793][SQL] Prohibit saving of day-time and year-month intervals
### What changes were proposed in this pull request?
For all built-in datasources, prohibit saving of year-month and day-time intervals that were introduced by SPARK-27793. We plan to support saving of such types at the milestone 2, see SPARK-27790.

### Why are the changes needed?
To improve user experience with Spark SQL, and print nicer error message. Current error message might confuse users:
```
scala> Seq(java.time.Period.ofMonths(1)).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
21/03/18 22:44:35 ERROR FileFormatWriter: Aborting job 8de402d7-ab69-4dc0-aa8e-14ef06bd2d6b.
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 1) (192.168.1.66 executor driver): org.apache.spark.SparkException: Task failed while writing rows.
	at org.apache.spark.sql.errors.QueryExecutionErrors$.taskFailedWhileWritingRowsError(QueryExecutionErrors.scala:418)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:298)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:211)
	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:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
Caused by: java.lang.RuntimeException: Failed to convert value 1 (class of class java.lang.Integer}) with the type of YearMonthIntervalType to JSON.
	at scala.sys.package$.error(package.scala:30)
	at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$23(JacksonGenerator.scala:179)
	at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$23$adapted(JacksonGenerator.scala:176)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above:
```
scala> Seq(java.time.Period.ofMonths(1)).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
```

### How was this patch tested?
1. Checked nested intervals:
```
scala> spark.range(1).selectExpr("""struct(timestamp'2021-01-02 00:01:02' - timestamp'2021-01-01 00:00:00')""").write.mode("overwrite").parquet("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
scala> Seq(Seq(java.time.Period.ofMonths(1))).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
```
2. By running existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2DataFrameSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
```

Closes #31884 from MaxGekk/ban-save-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-19 18:47:53 +03:00
Hongyi Zhang 6f89cdfb0c [SPARK-34798][SQL][TESTS] Fix incorrect join condition
### What changes were proposed in this pull request?

join condition 'a.attr == 'c.attr check the reference of  these 2 objects which will always returns false. we need to use === instead

### Why are the changes needed?

Although this join condition always false doesn't break the test but it is not what we expected. We should fix it to avoid future confusing

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

No

### How was this patch tested?

UT

Closes #31890 from opensky142857/SPARK-34798.

Authored-by: Hongyi Zhang <hongyzhang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-19 23:35:15 +08:00
Wenchen Fan 4b4f8e2a25 [SPARK-34558][SQL][FOLLOWUP] Use final Hadoop conf to instantiate FileSystem in SharedState
### What changes were proposed in this pull request?

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

https://github.com/apache/spark/pull/31671 has an unexpected behavior change that it uses a different Hadoop conf (`sparkContext.hadoopConfiguration`) to instantiate `FileSystem`, which is used to qualify the warehouse path. Before https://github.com/apache/spark/pull/31671 , the Hadoop conf to instantiate `FileSystem` is `session.sessionState.newHadoopConf()`.

More specifically, `session.sessionState.newHadoopConf()` has more conf entries:
1. it includes configs from `SharedState.initialConfigs`
2. in includes configs from `sparkContext.conf`

This PR updates `SharedState` to use the final Hadoop conf to instantiate `FileSystem`.

### Why are the changes needed?

fix behavior change

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

yes, the behavior will be the same before https://github.com/apache/spark/pull/31671

### How was this patch tested?

manually check the log of `FileSystem` and verify the passed in configs.

Closes #31868 from cloud-fan/followup.

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-03-19 22:02:15 +08:00
ulysses-you 58509565f8 [SPARK-34772][SQL] RebaseDateTime loadRebaseRecords should use Spark classloader instead of context
### What changes were proposed in this pull request?

Change context classloader to Spark classloader at `RebaseDateTime.loadRebaseRecords`

### Why are the changes needed?

With custom `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars`.

Spark would use date formatter in `HiveShim` that convert `date` to `string`, if we set `spark.sql.legacy.timeParserPolicy=LEGACY` and the partition type is `date` the `RebaseDateTime` code will be invoked. At that moment, if `RebaseDateTime` is initialized the first time then context class loader is `IsolatedClientLoader`. Such error msg would throw:

```
java.lang.IllegalArgumentException: argument "src" is null
  at com.fasterxml.jackson.databind.ObjectMapper._assertNotNull(ObjectMapper.java:4413)
  at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3157)
  at com.fasterxml.jackson.module.scala.ScalaObjectMapper.readValue(ScalaObjectMapper.scala:187)
  at com.fasterxml.jackson.module.scala.ScalaObjectMapper.readValue$(ScalaObjectMapper.scala:186)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$$anon$1.readValue(RebaseDateTime.scala:267)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$.loadRebaseRecords(RebaseDateTime.scala:269)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$.<init>(RebaseDateTime.scala:291)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$.<clinit>(RebaseDateTime.scala)
  at org.apache.spark.sql.catalyst.util.DateTimeUtils$.toJavaDate(DateTimeUtils.scala:109)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format(DateFormatter.scala:95)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format$(DateFormatter.scala:94)
  at org.apache.spark.sql.catalyst.util.LegacySimpleDateFormatter.format(DateFormatter.scala:138)
  at org.apache.spark.sql.hive.client.Shim_v0_13$ExtractableLiteral$1$.unapply(HiveShim.scala:661)
  at org.apache.spark.sql.hive.client.Shim_v0_13.convert$1(HiveShim.scala:785)
  at org.apache.spark.sql.hive.client.Shim_v0_13.$anonfun$convertFilters$4(HiveShim.scala:826)
```

```
java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.sql.catalyst.util.RebaseDateTime$
  at org.apache.spark.sql.catalyst.util.DateTimeUtils$.toJavaDate(DateTimeUtils.scala:109)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format(DateFormatter.scala:95)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format$(DateFormatter.scala:94)
  at org.apache.spark.sql.catalyst.util.LegacySimpleDateFormatter.format(DateFormatter.scala:138)
  at org.apache.spark.sql.hive.client.Shim_v0_13$ExtractableLiteral$1$.unapply(HiveShim.scala:661)
  at org.apache.spark.sql.hive.client.Shim_v0_13.convert$1(HiveShim.scala:785)
  at org.apache.spark.sql.hive.client.Shim_v0_13.$anonfun$convertFilters$4(HiveShim.scala:826)
  at scala.collection.immutable.Stream.flatMap(Stream.scala:493)
  at org.apache.spark.sql.hive.client.Shim_v0_13.convertFilters(HiveShim.scala:826)
  at org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:848)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getPartitionsByFilter$1(HiveClientImpl.scala:749)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:291)
  at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:224)
  at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:223)
  at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:273)
  at org.apache.spark.sql.hive.client.HiveClientImpl.getPartitionsByFilter(HiveClientImpl.scala:747)
  at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$listPartitionsByFilter$1(HiveExternalCatalog.scala:1273)
```

The reproduce steps:
1. `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars`.
2. `CREATE TABLE t (c int) PARTITIONED BY (p date)`
3. `SET spark.sql.legacy.timeParserPolicy=LEGACY`
4. `SELECT * FROM t WHERE p='2021-01-01'`

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

Yes, bug fix.

### How was this patch tested?

pass `org.apache.spark.sql.catalyst.util.RebaseDateTimeSuite` and add new unit test to `HiveSparkSubmitSuite.scala`.

Closes #31864 from ulysses-you/SPARK-34772.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-19 12:51:43 +08:00
Max Gekk a48b2086dd [SPARK-34761][SQL] Support add/subtract of a day-time interval to/from a timestamp
### What changes were proposed in this pull request?
Support `timestamp +/- day-time interval`. In the PR, I propose to extend the `TimeAdd` expression and support `DayTimeIntervalType` as the `interval` parameter. The expression invokes the new method `DateTimeUtils.timestampAddDayTime()` which splits the input day-time interval to `days` and `microsecond adjustment` of a day, and adds `days` (and the microseconds) to a local timestamp derived from the given timestamp at the given time zone.  The resulted local timestamp is converted back to the offset in microseconds since the epoch.

Also I updated the rules that handle `CalendarIntervalType` and produce `TimeAdd` to take into account new type `DateTimeIntervalType` for the `interval` parameter of `TimeAdd`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over timestamps and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/111081674-865d4900-8515-11eb-86c8-3538ecaf4804.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31855 from MaxGekk/timestamp-add-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-19 04:02:34 +00:00
Karuppayya Rajendran 0a58029d52 [SPARK-31897][SQL] Enable codegen for GenerateExec
### What changes were proposed in this pull request?
Enabling codegen for GenerateExec

### Why are the changes needed?
To leverage code generation for Generators

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

### How was this patch tested?
- UT tests added

### Benchmark
```
case class Data(value1: Float, value2: Map[String, String], value3: String)
val path = "<path>"

val numRecords = Seq(10000000, 100000000)
numRecords.map {
  recordCount =>
    import java.util.concurrent.TimeUnit.NANOSECONDS

    val srcDF = spark.range(recordCount).map {
      x => Data(x.toFloat, Map(x.toString -> x.toString ), s"value3$x")
    }.select($"value1", explode($"value2"), $"value3")
    val start = System.nanoTime()
    srcDF
      .write
      .mode("overwrite")
      .parquet(s"$path/$recordCount")
    val end = System.nanoTime()
    val diff = end - start
    (recordCount, NANOSECONDS.toMillis(diff))
}
```
**With codegen**:
```
res0: Seq[(Int, Long)] = List((10000000,13989), (100000000,129625))
```
**Without codegen**:
```
res0: Seq[(Int, Long)] = List((10000000,15736), (100000000,150399))
```

Closes #28715 from karuppayya/SPARK-31897.

Lead-authored-by: Karuppayya Rajendran <karuppayya1990@gmail.com>
Co-authored-by: Karuppayya Rajendran <karuppayya.rajendran@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-18 20:50:28 -07:00
Kousuke Saruta 07ee73234f [SPARK-34747][SQL][DOCS] Add virtual operators to the built-in function document
### What changes were proposed in this pull request?

This PR fix an issue that virtual operators (`||`, `!=`, `<>`, `between` and `case`) are absent from the Spark SQL Built-in functions document.

### Why are the changes needed?

The document should explain about all the supported built-in operators.

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

No.

### How was this patch tested?

Built the document with `SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_PYTHONDOC=1 bundler exec jekyll build` and then, confirmed the document.

![neq1](https://user-images.githubusercontent.com/4736016/111192859-e2e76380-85fc-11eb-89c9-75916a5e856a.png)
![neq2](https://user-images.githubusercontent.com/4736016/111192874-e7ac1780-85fc-11eb-9a9b-c504265b373f.png)
![between](https://user-images.githubusercontent.com/4736016/111192898-eda1f880-85fc-11eb-992d-cf80c544ec27.png)
![case](https://user-images.githubusercontent.com/4736016/111192918-f266ac80-85fc-11eb-9306-5dbc413a0cdb.png)
![double_pipe](https://user-images.githubusercontent.com/4736016/111192952-fb577e00-85fc-11eb-932e-385e5c2a5205.png)

Closes #31841 from sarutak/builtin-op-doc.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-19 10:19:26 +09:00
Cheng Su 8207e2f65c [SPARK-34781][SQL] Eliminate LEFT SEMI/ANTI joins to its left child side in AQE
### What changes were proposed in this pull request?

In `EliminateJoinToEmptyRelation.scala`, we can extend it to cover more cases for LEFT SEMI and LEFT ANI joins:

* Join is left semi join, join right side is non-empty and condition is empty. Eliminate join to its left side.
* Join is left anti join, join right side is empty. Eliminate join to its left side.

Given we eliminate join to its left side here, renaming the current optimization rule to `EliminateUnnecessaryJoin` instead.
In addition, also change to use `checkRowCount()` to check run time row count, instead of using `EmptyHashedRelation`. So this can cover `BroadcastNestedLoopJoin` as well. (`BroadcastNestedLoopJoin`'s broadcast side is `Array[InternalRow]`, not `HashedRelation`).

### Why are the changes needed?

Cover more join cases, and improve query performance for affected queries.

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

No.

### How was this patch tested?

Added unit tests in `AdaptiveQueryExecSuite.scala`.

Closes #31873 from c21/aqe-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-19 09:41:52 +09:00
yi.wu d99135b66a [SPARK-34741][SQL] MergeIntoTable should avoid ambiguous reference in UpdateAction
### What changes were proposed in this pull request?

This PR proposes to deduplicate the source table when there're conflicting attributes between the target table and the source table.

### Why are the changes needed?

When resolving the `UpdateAction`, which could reference attributes from both target and source tables,  Spark should know clearly where the attribute comes from when there're conflicting attributes instead of picking up a random one.

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

No.

### How was this patch tested?

Added a unit test and updated existing tests.

Closes #31835 from Ngone51/dedup-MergeIntoTable.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-18 15:54:41 +08:00
Luan 25e7d1ceee [SPARK-34728][SQL] Remove all SQLConf.get if extends from SQLConfHelper
### What changes were proposed in this pull request?

Remove all SQLConf.get to conf if extends from SQLConfHelper

### Why are the changes needed?

Clean up code.

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

No

### How was this patch tested?

Existing unit tests.

Closes #31822 from leoluan2009/SPARK-34728.

Authored-by: Luan <luanxuedong2009@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 15:04:41 +09:00
yi.wu 4d90c5dc0e [SPARK-34087][SQL] Fix memory leak of ExecutionListenerBus
### What changes were proposed in this pull request?

This PR proposes an alternative way to fix the memory leak of `ExecutionListenerBus`, which would automatically clean them up.

Basically, the idea is to add `registerSparkListenerForCleanup` to `ContextCleaner`, so we can remove the `ExecutionListenerBus` from `LiveListenerBus` when the `SparkSession` is GC'ed.

On the other hand, to make the `SparkSession` GC-able, we need to get rid of the reference of `SparkSession` in `ExecutionListenerBus`. Therefore, we introduced the `sessionUUID`, which is a unique identifier for SparkSession, to replace the  `SparkSession` object.

Note that, the proposal wouldn't take effect when `spark.cleaner.referenceTracking=false` since it depends on `ContextCleaner`.

### Why are the changes needed?

Fix the memory leak caused by `ExecutionListenerBus` mentioned in SPARK-34087.

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

Yes, save memory for users.

### How was this patch tested?

Added unit test.

Closes #31839 from Ngone51/fix-mem-leak-of-ExecutionListenerBus.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 13:27:03 +09:00
Kousuke Saruta c5cadfefdf [SPARK-34762][BUILD] Fix the build failure with Scala 2.13 which is related to commons-cli
### What changes were proposed in this pull request?

This PR fixes the build failure with Scala 2.13 which is related to `commons-cli`.
The last few days, build with Scala 2.13 on GA continues to fail and the error message says like as follows.
```
[error] /home/runner/work/spark/spark/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java:26:1:  error: package org.apache.commons.cli does not exist
1278[error] import org.apache.commons.cli.GnuParser;
```
The reason is that `mvn help` in `change-scala-version.sh` downloads the POM file of `commons-cli` but doesn't download the JAR file, leading the build failure.

This PR also adds `commons-cli` to the dependencies explicitly because HiveThriftServer depends on it.
### Why are the changes needed?

Expect to fix the build failure with Scala 2.13.

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

No.

### How was this patch tested?

I confirmed that build successfully finishes with Scala 2.13 on my laptop.
```
find ~/.m2 -name commons-cli -exec rm -rf {} \;
find ~/.ivy2 -name commons-cli -exec rm -rf {} \;
find ~/.cache/ -name commons-cli -exec rm -rf {} \; // For Linux
find ~/Library/Caches -name commons-cli -exec rm -rf {} \; // For macOS

dev/change-scala-version 2.13
./build/sbt -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pdocker-integration-tests -Pkubernetes-integration-tests -Pspark-ganglia-lgpl -Pscala-2.13 clean compile test:compile
```

Closes #31862 from sarutak/commons-cli.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 12:31:50 +09:00
gengjiaan 569fb133d0 [SPARK-33602][SQL] Group exception messages in execution/datasources
### What changes were proposed in this pull request?
This PR group exception messages in `/core/src/main/scala/org/apache/spark/sql/execution/datasources`.

### 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 #31757 from beliefer/SPARK-33602.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 14:04:02 +00:00
Wenchen Fan 9f7b0a035b [SPARK-34758][SQL] Simplify Analyzer.resolveLiteralFunction
### What changes were proposed in this pull request?

This PR simplifies `Analyzer.resolveLiteralFunction` to always create the `Alias`. The caller side will remove the `Alias` if it's not necessary.

### Why are the changes needed?

code simplification.

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

no

### How was this patch tested?

existing tests

Closes #31844 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-17 21:26:44 +09:00
Wenchen Fan bf4570b43d [SPARK-34749][SQL] Simplify ResolveCreateNamedStruct
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31808 and simplifies its fix to one line (excluding comments).

### Why are the changes needed?

code simplification

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

no

### How was this patch tested?

N/A

Closes #31843 from cloud-fan/simplify.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-17 21:21:54 +09:00
ulysses-you 48637a9d43 [SPARK-34766][SQL] Do not capture maven config for views
### What changes were proposed in this pull request?

Skip capture maven repo config for views.

### Why are the changes needed?

Due to the bad network, we always use the thirdparty maven repo to run test. e.g.,
```
build/sbt "test:testOnly *SQLQueryTestSuite" -Dspark.sql.maven.additionalRemoteRepositories=xxxxx
```

It's failed with such error msg
```
[info] - show-tblproperties.sql *** FAILED *** (128 milliseconds)
[info] show-tblproperties.sql
[info] Expected "...rredTempViewNames [][]", but got "...rredTempViewNames [][
[info] view.sqlConfig.spark.sql.maven.additionalRemoteRepositories xxxxx]" Result did not match for query #6
[info] SHOW TBLPROPERTIES view (SQLQueryTestSuite.scala:464)
```

It's not necessary to capture the maven config to view since it's a session level config.
 

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

No.

### How was this patch tested?

manual test pass
```
build/sbt "test:testOnly *SQLQueryTestSuite" -Dspark.sql.maven.additionalRemoteRepositories=xxx
```

Closes #31856 from ulysses-you/skip-maven-config.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-17 20:12:18 +08:00
HyukjinKwon 385f1e8f5d [SPARK-34768][SQL] Respect the default input buffer size in Univocity
### What changes were proposed in this pull request?

This PR proposes to follow Univocity's input buffer.

### Why are the changes needed?

- Firstly, it's best to trust their judgement on the default values. Also 128 is too low.
- Default values arguably have more test coverage in Univocity.
- It will also fix https://github.com/uniVocity/univocity-parsers/issues/449
- ^ is a regression compared to Spark 2.4

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

No. In addition, It fixes a regression.

### How was this patch tested?

Manually tested, and added a unit test.

Closes #31858 from HyukjinKwon/SPARK-34768.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-17 19:55:49 +09:00
Wenchen Fan 1a4971d8a1 [SPARK-34770][SQL] InMemoryCatalog.tableExists should not fail if database doesn't exist
### What changes were proposed in this pull request?

This PR updates `InMemoryCatalog.tableExists` to return false if database doesn't exist, instead of failing. The new behavior is consistent with `HiveExternalCatalog` which is used in production, so this bug mostly only affects tests.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

a new test

Closes #31860 from cloud-fan/catalog.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 16:36:50 +08:00
Kent Yao 115f777cb0 [SPARK-21449][SQL][FOLLOWUP] Avoid log undesirable IllegalStateException when state close
### What changes were proposed in this pull request?

`TmpOutputFile` and `TmpErrOutputFile`  are registered in `o.a.h.u.ShutdownHookManager `during creatation. The `state.close()` will delete them if they are not null and try remove them from the `o.a.h.u.ShutdownHookManager` which causes IllegalStateException when we call it in our ShutdownHookManager too.
In this PR, we delete them ahead with a high priority hook in Spark and set them to null to bypass the deletion and canceling in `state.close()`

### Why are the changes needed?

W/ or w/o this PR, the deletion of these files is not affected, we just mute an undesirable error log here.

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

no, this is a follow-up

### How was this patch tested?

#### the undesirable gone
```scala
spark-sql> 21/03/16 18:41:31 ERROR Utils: Uncaught exception in thread shutdown-hook-0
java.lang.IllegalStateException: Shutdown in progress, cannot cancel a deleteOnExit
	at org.apache.hive.common.util.ShutdownHookManager.cancelDeleteOnExit(ShutdownHookManager.java:106)
	at org.apache.hadoop.hive.common.FileUtils.deleteTmpFile(FileUtils.java:861)
	at org.apache.hadoop.hive.ql.session.SessionState.deleteTmpErrOutputFile(SessionState.java:325)
	at org.apache.hadoop.hive.ql.session.SessionState.dropSessionPaths(SessionState.java:829)
	at org.apache.hadoop.hive.ql.session.SessionState.close(SessionState.java:1585)
	at org.apache.hadoop.hive.cli.CliSessionState.close(CliSessionState.java:66)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:172)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1994)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$1(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at scala.util.Try$.apply(Try.scala:213)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:188)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:178)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	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)
(python)  ✘ kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316  cd ..
(python)  kentyaohulk  ~/Downloads/spark  tar zxf spark-3.2.0-SNAPSHOT-bin-20210316.tgz
(python)  kentyaohulk  ~/Downloads/spark  cd -
~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316
(python)  kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316  bin/spark-sql --conf spark.local.dir=./local --conf spark.hive.exec.local.scratchdir=./local
21/03/16 18:42:15 WARN Utils: Your hostname, hulk.local resolves to a loopback address: 127.0.0.1; using 10.242.189.214 instead (on interface en0)
21/03/16 18:42:15 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
21/03/16 18:42:15 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
21/03/16 18:42:16 WARN SparkConf: Note that spark.local.dir will be overridden by the value set by the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone/kubernetes and LOCAL_DIRS in YARN).
21/03/16 18:42:18 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
21/03/16 18:42:18 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
21/03/16 18:42:19 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
21/03/16 18:42:19 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore kentyao127.0.0.1
Spark master: local[*], Application Id: local-1615891336877
spark-sql> %
```

#### and the deletion is still fine

```shell
kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316 
ls -al local
total 0
drwxr-xr-x   7 kentyao  staff  224  3 16 18:42 .
drwxr-xr-x  19 kentyao  staff  608  3 16 18:42 ..
drwx------   2 kentyao  staff   64  3 16 18:42 16cc5238-e25e-4c0f-96ef-0c4bdecc7e51
-rw-r--r--   1 kentyao  staff    0  3 16 18:42 16cc5238-e25e-4c0f-96ef-0c4bdecc7e51219959790473242539.pipeout
-rw-r--r--   1 kentyao  staff    0  3 16 18:42 16cc5238-e25e-4c0f-96ef-0c4bdecc7e518816377057377724129.pipeout
drwxr-xr-x   2 kentyao  staff   64  3 16 18:42 blockmgr-37a52ad2-eb56-43a5-8803-8f58d08fe9ad
drwx------   3 kentyao  staff   96  3 16 18:42 spark-101971df-f754-47c2-8764-58c45586be7e
 kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316  ls -al local
total 0
drwxr-xr-x   2 kentyao  staff   64  3 16 19:22 .
drwxr-xr-x  19 kentyao  staff  608  3 16 18:42 ..
 kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316 
```

Closes #31850 from yaooqinn/followup.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-17 15:21:23 +08:00
Yuming Wang c234c5b5f1 [SPARK-34575][SQL] Push down limit through window when partitionSpec is empty
### What changes were proposed in this pull request?

Push down limit through `Window` when the partitionSpec of all window functions is empty and the same order is used. This is a real case from production:

![image](https://user-images.githubusercontent.com/5399861/109457143-3900c680-7a95-11eb-9078-806b041175c2.png)

This pr support 2 cases:
1. All window functions have same orderSpec:
   ```sql
   SELECT *, ROW_NUMBER() OVER(ORDER BY a) AS rn, RANK() OVER(ORDER BY a) AS rk FROM t1 LIMIT 5;
   == Optimized Logical Plan ==
   Window [row_number() windowspecdefinition(a#9L ASC NULLS FIRST, specifiedwindowframe(RowFrame,          unboundedpreceding$(), currentrow$())) AS rn#4, rank(a#9L) windowspecdefinition(a#9L ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#5], [a#9L ASC NULLS FIRST]
   +- GlobalLimit 5
      +- LocalLimit 5
         +- Sort [a#9L ASC NULLS FIRST], true
            +- Relation default.t1[A#9L,B#10L,C#11L] parquet
   ```
2. There is a window function with a different orderSpec:
   ```sql
   SELECT a, ROW_NUMBER() OVER(ORDER BY a) AS rn, RANK() OVER(ORDER BY b DESC) AS rk FROM t1 LIMIT 5;
   == Optimized Logical Plan ==
   Project [a#9L, rn#4, rk#5]
   +- Window [rank(b#10L) windowspecdefinition(b#10L DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#5], [b#10L DESC NULLS LAST]
      +- GlobalLimit 5
         +- LocalLimit 5
            +- Sort [b#10L DESC NULLS LAST], true
               +- Window [row_number() windowspecdefinition(a#9L ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#4], [a#9L ASC NULLS FIRST]
                  +- Project [a#9L, b#10L]
                     +- Relation default.t1[A#9L,B#10L,C#11L] parquet
   ```

### Why are the changes needed?

Improve query performance.

```scala
spark.range(500000000L).selectExpr("id AS a", "id AS b").write.saveAsTable("t1")
spark.sql("SELECT *, ROW_NUMBER() OVER(ORDER BY a) AS rowId FROM t1 LIMIT 5").show
```

Before this pr | After this pr
-- | --
![image](https://user-images.githubusercontent.com/5399861/109456919-c68fe680-7a94-11eb-89ca-67ec03267158.png) | ![image](https://user-images.githubusercontent.com/5399861/109456927-cd1e5e00-7a94-11eb-9866-d76b2665caea.png)

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

No.

### How was this patch tested?

Unit test.

Closes #31691 from wangyum/SPARK-34575.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 07:16:10 +00:00
Gengliang Wang 143303147b [SPARK-34742][SQL] ANSI mode: Abs throws exception if input is out of range
### What changes were proposed in this pull request?

For the following cases, ABS should throw exceptions since the results are out of the range of the result data types in ANSI mode.
```
SELECT abs(${Int.MinValue});
SELECT abs(${Long.MinValue});
```
### Why are the changes needed?

Better ANSI compliance

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

Yes, Abs throws an exception if input is out of range in ANSI mode

### How was this patch tested?

Unit test

Closes #31836 from gengliangwang/ansiAbs.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 06:57:25 +00:00
Terry Kim 387d866244 [SPARK-34699][SQL] 'CREATE OR REPLACE TEMP VIEW USING' should uncache correctly
### What changes were proposed in this pull request?

This PR proposes:
  1. `CREATE OR REPLACE TEMP VIEW USING` should use `TemporaryViewRelation` to store temp views.
  2. By doing #1, it fixes the issue where the temp view being replaced is not uncached.

### Why are the changes needed?

This is a part of an ongoing work to wrap all the temporary views with `TemporaryViewRelation`: [SPARK-34698](https://issues.apache.org/jira/browse/SPARK-34698).

This also fixes a bug where the temp view being replaced is not uncached.

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

Yes, the temp view being replaced with `CREATE OR REPLACE TEMP VIEW USING` is correctly uncached if the temp view is cached.

### How was this patch tested?

Added new tests.

Closes #31825 from imback82/create_temp_view_using.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 06:04:07 +00:00
Wenchen Fan af553735b1 [SPARK-34504][SQL] Avoid unnecessary resolving of SQL temp views for DDL commands
### What changes were proposed in this pull request?

For DDL commands like DROP VIEW, they don't really need to resolve the view (parse and analyze the view SQL text), they just need to get the view metadata.

This PR fixes the rule `ResolveTempViews` to only resolve the temp view for `UnresolvedRelation`. This also fixes a bug for DROP VIEW, as previously it tried to resolve the view and failed to drop invalid views.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

new test

Closes #31853 from cloud-fan/view-resolve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 11:16:51 +08:00
Wenchen Fan cef6650048 Revert "[SPARK-33428][SQL] Conv UDF use BigInt to avoid Long value overflow"
This reverts commit 5f9a7fea06.
2021-03-16 13:56:50 +08:00
Cheng Su bb05dc91f0 [SPARK-34729][SQL][FOLLOWUP] Broadcast nested loop join to use executeTake instead of execute
### What changes were proposed in this pull request?

This is a followup minor change from https://github.com/apache/spark/pull/31821#discussion_r594110622 , where we change from using `execute()` to `executeTake()`. Performance-wise there's no difference. We are just using a different API to be aligned with code path of `Dataset`.

### Why are the changes needed?

To align with other code paths in SQL/Dataset.

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

No.

### How was this patch tested?

Existing unit tests same as https://github.com/apache/spark/pull/31821 .

Closes #31845 from c21/join-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-15 19:48:45 -07:00
Kent Yao 202529ef23 [SPARK-21449][SPARK-23745][SQL] add ShutdownHook to cloes HiveClient's SessionState to delete residual dirs
### What changes were proposed in this pull request?

We initialized a Hive `SessionState` to interact with the external hive metastore server but left it behind after we finished.

We should close the metastore client explicitly in case of connection leaks with HMS
and we should trigger the `SessionState` to close itself to clean the residual dirs to fix issues reported by SPARK-21449 and SPARK-23745.

`hive.downloaded.resources.dir` contains transient files, such as UDF jars, it will not be used anymore after spark applications exit.

### Why are the changes needed?

1. prevent potential metastore client leak

2. clean `hive.downloaded.resources.dir`

```
    DOWNLOADED_RESOURCES_DIR("hive.downloaded.resources.dir", "${system:java.io.tmpdir}" + File.separator + "${hive.session.id}_resources", "Temporary local directory for added resources in the remote file system."),

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

no

### How was this patch tested?

passing jenkins and verify locally

Closes #31833 from yaooqinn/SPARK-21449-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-16 10:37:40 +08:00
Dongjoon Hyun 0a70dff066 [MINOR][SQL] Remove unused variable in NewInstance.constructor
### What changes were proposed in this pull request?

This PR removes one unused variable in `NewInstance.constructor`.

### Why are the changes needed?

This looks like a variable for debugging at the initial commit of SPARK-23584 .
- 1b08c4393c (diff-2a36e31684505fd22e2d12a864ce89fd350656d716a3f2d7789d2cdbe38e15fbR461)

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31838 from dongjoon-hyun/minor-object.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-15 18:49:54 -07:00
Max Gekk 9809a2f1c5 [SPARK-34739][SQL] Support add/subtract of a year-month interval to/from a timestamp
### What changes were proposed in this pull request?
Support `timestamp +/- year-month interval`. In the PR, I propose to introduce new binary expression `TimestampAddYMInterval` similarly to `DateAddYMInterval`. It invokes new method `timestampAddMonths` from `DateTimeUtils` by passing a timestamp as an offset in microseconds since the epoch, amount of months from the giveb year-month interval, and the time zone ID in which the operation is performed. The `timestampAddMonths()` method converts the input microseconds to a local timestamp, adds months to it, and converts the results back to an instant in microseconds at the given time zone.

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over timestamps and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/111081674-865d4900-8515-11eb-86c8-3538ecaf4804.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31832 from MaxGekk/timestamp-add-year-month-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-15 14:36:12 +03:00
Dongjoon Hyun 363a7f0722 [SPARK-34743][SQL][TESTS] ExpressionEncoderSuite should use deepEquals when we expect array of array
### What changes were proposed in this pull request?

This PR aims to make `ExpressionEncoderSuite` to use `deepEquals` instead of `equals` when `input` is `array of array`.

This comparison code itself was added by SPARK-11727 at Apache Spark 1.6.0.

### Why are the changes needed?

Currently, the interpreted mode fails for `array of array` because the following line is used.
```
Arrays.equals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]])
```

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

No. This is a test-only PR.

### How was this patch tested?

Pass the existing CIs.

Closes #31837 from dongjoon-hyun/SPARK-34743.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-15 02:30:54 -07:00
Wenchen Fan be888b27ed [SPARK-34639][SQL] Always remove unnecessary Alias in Analyzer.resolveExpression
### What changes were proposed in this pull request?

In `Analyzer.resolveExpression`, we have a parameter to decide if we should remove unnecessary `Alias` or not. This is over complicated and we can always remove unnecessary `Alias`.

This PR simplifies this part and removes the parameter.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31758 from cloud-fan/resolve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-15 09:22:36 +00:00
Cheng Su a0f3b72e1c [SPARK-34729][SQL] Faster execution for broadcast nested loop join (left semi/anti with no condition)
### What changes were proposed in this pull request?

For `BroadcastNestedLoopJoinExec` left semi and left anti join without condition. If we broadcast left side. Currently we check whether every row from broadcast side has a match or not by [iterating broadcast side a lot of time](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala#L256-L275). This is unnecessary and very inefficient when there's no condition, as we only need to check whether stream side is empty or not. Create this PR to add the optimization. This can boost the affected query execution performance a lot.

In addition, create a common method `getMatchedBroadcastRowsBitSet()` shared by several methods.
Refactor `defaultJoin()` to move
* left semi and left anti join related logic to `leftExistenceJoin`
* existence join related logic to `existenceJoin`.

After this, `defaultJoin()` holds logic only for outer join (left outer, right outer and full outer), which is much easier to read from my own opinion.

### Why are the changes needed?

Improve the affected query performance a lot.
Test with a simple query by modifying `JoinBenchmark.scala` locally:

```
val N = 20 << 20
val M = 1 << 4
val dim = broadcast(spark.range(M).selectExpr("id as k"))
val df = dim.join(spark.range(N), Seq.empty, "left_semi")
df.noop()
```

See >30x run time improvement. Note the stream side is only `spark.range(N)`. For complicated query with non-trivial stream side, the saving would be much more.

```
Running benchmark: broadcast nested loop left semi join
  Running case: broadcast nested loop left semi join optimization off
  Stopped after 2 iterations, 3163 ms
  Running case: broadcast nested loop left semi join optimization on
  Stopped after 5 iterations, 366 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
broadcast nested loop left semi join:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------------------
broadcast nested loop left semi join optimization off           1568           1582          19         13.4          74.8       1.0X
broadcast nested loop left semi join optimization on              46             73          18        456.0           2.2      34.1X
```

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

No.

### How was this patch tested?

Added unit test in `ExistenceJoinSuite.scala`.

Closes #31821 from c21/nested-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-14 23:51:36 -07:00
yangjie01 e757091820 [SPARK-34722][CORE][SQL][TEST] Clean up deprecated API usage related to JUnit4
### What changes were proposed in this pull request?
The main change of this pr as follows:

- Use `org.junit.Assert.assertThrows(String, Class, ThrowingRunnable)` method instead of  `ExpectedException.none()`
- Use `org.hamcrest.MatcherAssert.assertThat()` method instead of   `org.junit.Assert.assertThat(T, org.hamcrest.Matcher<? super T>)`

### Why are the changes needed?
Clean up deprecated API usage

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

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

Closes #31815 from LuciferYang/SPARK-34722.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-14 23:33:03 -07:00
Max Gekk 7aaed76125 [SPARK-34737][SQL] Cast input float to double in TIMESTAMP_SECONDS
### What changes were proposed in this pull request?
In the PR, I propose to cast the input float to double in the `SecondsToTimestamp` expression in the same way as in the `Cast` expression.

### Why are the changes needed?
To have the same results from `CAST(<float> AS TIMESTAMP)` and from `TIMESTAMP_SECONDS`:
```sql
spark-sql> SELECT CAST(16777215.0f AS TIMESTAMP);
1970-07-14 07:20:15
spark-sql> SELECT TIMESTAMP_SECONDS(16777215.0f);
1970-07-14 07:20:14.951424
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
spark-sql> SELECT TIMESTAMP_SECONDS(16777215.0f);
1970-07-14 07:20:15
```

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *DateExpressionsSuite"
```

Closes #31831 from MaxGekk/adjust-SecondsToTimestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-15 10:05:59 +09:00
Max Gekk e0a1399bd7 [SPARK-34727][SQL] Fix discrepancy in casting float to timestamp
### What changes were proposed in this pull request?
In non-ANSI mode, casting float to timestamp has different implementation for codegen on and off.

Codegen on:
1. Multiply float input by MICROS_PER_SECOND
2. Cast resulting float value to long

Codegen off:
1. CAST float input to double input
2. Multiply double input by MICROS_PER_SECOND
3. Cast resulting double value to long

In the PR, I propose to align to non-codegen code, and cast input float to double in codegen.

### Why are the changes needed?
This fixes the issue which is demonstrated by the code:
```sql
spark-sql> CREATE TEMP VIEW v1 AS SELECT 16777215.0f AS f;
spark-sql> SELECT * FROM v1;
1.6777215E7
spark-sql> SELECT CAST(f AS TIMESTAMP) FROM v1;
1970-07-14 07:20:15
spark-sql> CACHE TABLE v1;
spark-sql> SELECT * FROM v1;
1.6777215E7
spark-sql> SELECT CAST(f AS TIMESTAMP) FROM v1;
1970-07-14 07:20:14.951424
```
The result from the cached view **1970-07-14 07:20:14.951424** is different from un-cached view **1970-07-14 07:20:15**.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above outputs the same timestamp for the cached view:
```sql
spark-sql> CACHE TABLE v1;
spark-sql> SELECT * FROM v1;
1.6777215E7
spark-sql> SELECT CAST(f AS TIMESTAMP) FROM v1;
1970-07-14 07:20:15
```

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *CastSuite"
```

Closes #31819 from MaxGekk/fix-float-to-timestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-14 11:29:54 +09:00
Liang-Chi Hsieh 86baa36eeb [SPARK-34723][SQL] Correct parameter type for subexpression elimination under whole-stage
### What changes were proposed in this pull request?

This patch proposes to fix incorrect parameter type for subexpression elimination under whole-stage.

### Why are the changes needed?

If the parameter is a byte array, the subexpression elimination under wholestage codegen will use incorrect parameter type and cause compile error. Although Spark can automatically fallback to interpreted mode, we should fix it.

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

No

### How was this patch tested?

Manually test with customer application. Unit test.

Closes #31814 from viirya/SPARK-34723.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-13 00:05:41 -08:00
Max Gekk 4f1e434ec5 [SPARK-34721][SQL] Support add/subtract of a year-month interval to/from a date
### What changes were proposed in this pull request?
Support `date +/- year-month interval`. In the PR, I propose to re-use existing code from the `AddMonths` expression, and extract it to the common base class `AddMonthsBase`. That base class is used in new expression `DateAddYMInterval` and in the existing one `AddMonths` (the `add_months` function).

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over dates and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/110914390-5f412480-8327-11eb-9f8b-e92e73c0b9cd.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *ColumnExpressionSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
```

Closes #31812 from MaxGekk/date-add-year-month-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-12 14:35:56 +00:00
Dongjoon Hyun 9a7977933f [SPARK-34724][SQL] Fix Interpreted evaluation by using getMethod instead of getDeclaredMethod
### What changes were proposed in this pull request?

This bug was introduced by SPARK-23583 at Apache Spark 2.4.0.

This PR aims to use `getMethod` instead of `getDeclaredMethod`.
```scala
- obj.getClass.getDeclaredMethod(functionName, argClasses: _*)
+ obj.getClass.getMethod(functionName, argClasses: _*)
```

### Why are the changes needed?

`getDeclaredMethod` does not search the super class's method. To invoke `GenericArrayData.toIntArray`, we need to use `getMethod` because it's declared at the super class `ArrayData`.

```
[info] - encode/decode for array of int: [I74655d03 (interpreted path) *** FAILED *** (14 milliseconds)
[info]   Exception thrown while decoding
[info]   Converted: [0,1000000020,3,0,ffffff850000001f,4]
[info]   Schema: value#680
[info]   root
[info]   -- value: array (nullable = true)
[info]       |-- element: integer (containsNull = false)
[info]
[info]
[info]   Encoder:
[info]   class[value[0]: array<int>] (ExpressionEncoderSuite.scala:578)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Assertions.fail(Assertions.scala:949)
[info]   at org.scalatest.Assertions.fail$(Assertions.scala:945)
[info]   at org.scalatest.funsuite.AnyFunSuite.fail(AnyFunSuite.scala:1563)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$encodeDecodeTest$1(ExpressionEncoderSuite.scala:578)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.verifyNotLeakingReflectionObjects(ExpressionEncoderSuite.scala:656)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$testAndVerifyNotLeakingReflectionObjects$2(ExpressionEncoderSuite.scala:669)
[info]   at org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest.$anonfun$test$4(PlanTest.scala:50)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.withSQLConf(ExpressionEncoderSuite.scala:118)
[info]   at org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest.$anonfun$test$3(PlanTest.scala:50)
...
[info]   Cause: java.lang.RuntimeException: Error while decoding: java.lang.NoSuchMethodException: org.apache.spark.sql.catalyst.util.GenericArrayData.toIntArray()
[info] mapobjects(lambdavariable(MapObject, IntegerType, false, -1), assertnotnull(lambdavariable(MapObject, IntegerType, false, -1)), input[0, array<int>, true], None).toIntArray
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:186)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$encodeDecodeTest$1(ExpressionEncoderSuite.scala:576)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.verifyNotLeakingReflectionObjects(ExpressionEncoderSuite.scala:656)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$testAndVerifyNotLeakingReflectionObjects$2(ExpressionEncoderSuite.scala:669)
```

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

This causes a runtime exception when we use the interpreted mode.

### How was this patch tested?

Pass the modified unit test case.

Closes #31816 from dongjoon-hyun/SPARK-34724.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-12 21:30:46 +09:00
Kousuke Saruta 03dd33cc98 [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute, AttributeReference and Alias don't quote qualified names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `sql` method in the following classes which take qualified names don't quote the qualified names properly.

* UnresolvedAttribute
* AttributeReference
* Alias

One instance caused by this issue is reported in SPARK-34626.
```
UnresolvedAttribute("a" :: "b" :: Nil).sql
`a.b` // expected: `a`.`b`
```
And other instances are like as follows.
```
UnresolvedAttribute("a`b"::"c.d"::Nil).sql
a`b.`c.d` // expected: `a``b`.`c.d`

AttributeReference("a.b", IntegerType)(qualifier = "c.d"::Nil).sql
c.d.`a.b` // expected: `c.d`.`a.b`

Alias(AttributeReference("a", IntegerType)(), "b.c")(qualifier = "d.e"::Nil).sql
`a` AS d.e.`b.c` // expected: `a` AS `d.e`.`b.c`
```

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

New test.

Closes #31754 from sarutak/fix-qualified-names.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-12 02:58:46 +00:00
Max Gekk cebe2be221 [SPARK-34718][SQL] Assign pretty names to YearMonthIntervalType and DayTimeIntervalType
### What changes were proposed in this pull request?
In the PR, I propose to override the `typeName()` method in `YearMonthIntervalType` and `DayTimeIntervalType`, and assign them names according to the ANSI SQL standard:
<img width="836" alt="Screenshot 2021-03-11 at 17 29 04" src="https://user-images.githubusercontent.com/1580697/110802854-a54aa980-828f-11eb-956d-dd4fbf14aa72.png">
but keep the type name as singular according existing naming convention for other types.

### Why are the changes needed?
To improve Spark SQL user experience, and have readable types in error messages.

### 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 modified tests:
```
$ build/sbt "test:testOnly *ExpressionTypeCheckingSuite"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z windowFrameCoercion.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z literals.sql"
```

Closes #31810 from MaxGekk/interval-types-name.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 12:55:12 -08:00
Angerszhuuuu badca975af [SPARK-34712][SQL][TESTS] Refactor UT about hive build in version, avoid to change every time when upgrade hive version
### What changes were proposed in this pull request?
Use HiveUtils.buildinHiveVersion to replace correspoding Ut about hive version

### Why are the changes needed?
Refactor UT about hive build in version, avoid to change every time when upgrade hive version

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

### How was this patch tested?
Not need

Closes #31807 from AngersZhuuuu/SPARK-34712.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 12:52:29 -08:00
Wenchen Fan 6a42b633bf [SPARK-34713][SQL] Fix group by CreateStruct with ExtractValue
### What changes were proposed in this pull request?

This is a bug caused by https://issues.apache.org/jira/browse/SPARK-31670 . We remove the `Alias` when resolving column references in grouping expressions, which breaks `ResolveCreateNamedStruct`

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

new tests

Closes #31808 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 09:21:58 -08:00
Max Gekk d7bb327aee [SPARK-34695][SQL] Fix long overflow in conversion of minimum duration to microseconds
### What changes were proposed in this pull request?
In the PR, I propose to especially handle the amount of seconds `-9223372036855` in `IntervalUtils. durationToMicros()`. Starting from the amount (any durations with the second field < `-9223372036855`), input durations cannot fit to `Long` in the conversion to microseconds. For example, the amount of microseconds = `Long.MinValue = -9223372036854775808` can be represented in two forms:
1. seconds = -9223372036854, nanoAdjustment = -775808, or
2. seconds = -9223372036855, nanoAdjustment = +224192

And the method `Duration.ofSeconds()` produces the last form but such form causes overflow while converting `-9223372036855` seconds to microseconds.

In the PR, I propose to convert the second form to the first one if the second field of input duration is equal to `-9223372036855`.

### Why are the changes needed?
The changes fix the issue demonstrated by the code:
```scala
scala> durationToMicros(microsToDuration(Long.MinValue))
java.lang.ArithmeticException: long overflow
  at java.lang.Math.multiplyExact(Math.java:892)
  at org.apache.spark.sql.catalyst.util.IntervalUtils$.durationToMicros(IntervalUtils.scala:782)
  ... 49 elided
```
The `durationToMicros()` method cannot handle valid output of `microsToDuration()`.

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

### How was this patch tested?
By running new UT from `IntervalUtilsSuite`.

Closes #31799 from MaxGekk/fix-min-duration.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 15:21:15 +00:00
ulysses-you 744a73df9e [SPARK-34538][SQL] Hive Metastore support filter by not-in
### What changes were proposed in this pull request?

Add `Not(In)` and `Not(InSet)` pattern when convert filter to metastore.

### Why are the changes needed?

`NOT IN` is a useful condition to prune partition, it would be better to support it.

Technically, we can convert `c not in(x,y)` to `c != x and c != y`, then push it to metastore.

Avoid metastore overflow and respect the config `spark.sql.hive.metastorePartitionPruningInSetThreshold`, `Not(InSet)` won't push to metastore if it's value exceeds the threshold.

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

No.

### How was this patch tested?

Add test.

Closes #31646 from ulysses-you/SPARK-34538.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 15:19:47 +00:00
Cheng Su da086a8ea3 [SPARK-34702][SQL] Avoid unnecessary code generation in JoinCodegenSupport.genBuildSideVars
### What changes were proposed in this pull request?

As a followup from code review in https://github.com/apache/spark/pull/31736#discussion_r588134104 , for `JoinCodegenSupport.genBuildSideVars`, we only need to generate build side variables with default values for LEFT OUTER and RIGHT OUTER join, but not for other join types (i.e. LEFT SEMI and LEFT ANTI). Create this PR to clean up the code.

In addition, change `BroadcastNestedLoopJoinExec` unit test to cover both whole stage code-gen enabled and disabled. Harden the unit tests to exercise all code paths.

### Why are the changes needed?

Avoid unnecessary code generation.

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

No.

### How was this patch tested?

Existing unit tests.
* BHJ and SHJ inner join is covered in `InnerJoinSuite.scala`
* BHJ and SHJ left outer and right outer join are covered in `OuterJoinSuite.scala`
* BHJ and SHJ left semi, left anti and existence join are covered in `ExistenceJoinSuite.scala`

Closes #31802 from c21/join-codegen-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 13:29:45 +00:00
Kousuke Saruta fa1cf5c207 [SPARK-34697][SQL] Allow DESCRIBE FUNCTION and SHOW FUNCTIONS explain about || (string concatenation operator)
### What changes were proposed in this pull request?

This PR fixes the behavior of `SHOW FUNCTIONS` and `DESCRIBE FUNCTION` for the `||` operator.
The result of `SHOW FUNCTIONS` doesn't contains `||` and `DESCRIBE FUNCTION ||` says `Function: || not found.` even though `||` is supported.

### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

Confirmed manually with the following commands.
```
spark-sql> DESCRIBE FUNCTION ||;
Function: ||
Usage: expr1 || expr2 - Returns the concatenation of `expr1` and `expr2`.

spark-sql> SHOW FUNCTIONS;
!
!=
%

...

|
||
~
```

Closes #31800 from sarutak/fix-describe-concat-pipe.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-11 22:11:26 +09:00
Max Gekk 9d3d25bca4 [SPARK-34677][SQL] Support the +/- operators over ANSI SQL intervals
### What changes were proposed in this pull request?
Extend the `Add`, `Subtract` and `UnaryMinus` expression to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the `overflow` exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
To conform to the ANSI SQL standard which defines `-/+` over intervals:
<img width="822" alt="Screenshot 2021-03-09 at 21 59 22" src="https://user-images.githubusercontent.com/1580697/110523128-bd50ea80-8122-11eb-9982-782da0088d27.png">

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

### How was this patch tested?
By running new tests in the test suites:
```
$ build/sbt "test:testOnly *ArithmeticExpressionSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31789 from MaxGekk/add-subtruct-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 10:08:43 +00:00
Dongjoon Hyun 5c4d8f9538 [SPARK-34696][SQL][TESTS] Fix CodegenInterpretedPlanTest to generate correct test cases
### What changes were proposed in this pull request?

SPARK-23596 added `CodegenInterpretedPlanTest` at Apache Spark 2.4.0 in a wrong way because `withSQLConf` depends on the execution time `SQLConf.get` instead of `test` function declaration time. So, the following code executes the test twice without controlling the `CodegenObjectFactoryMode`. This PR aims to fix it correct and introduce a new function `testFallback`.

```scala
trait CodegenInterpretedPlanTest extends PlanTest {

   override protected def test(
       testName: String,
       testTags: Tag*)(testFun: => Any)(implicit pos: source.Position): Unit = {
     val codegenMode = CodegenObjectFactoryMode.CODEGEN_ONLY.toString
     val interpretedMode = CodegenObjectFactoryMode.NO_CODEGEN.toString

     withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) {
       super.test(testName + " (codegen path)", testTags: _*)(testFun)(pos)
     }
     withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) {
       super.test(testName + " (interpreted path)", testTags: _*)(testFun)(pos)
     }
   }
 }
```

### Why are the changes needed?

1. We need to use like the following.
```scala
super.test(testName + " (codegen path)", testTags: _*)(
   withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { testFun })(pos)
super.test(testName + " (interpreted path)", testTags: _*)(
   withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) { testFun })(pos)
```

2. After we fix this behavior with the above code, several test cases including SPARK-34596 and SPARK-34607 fail because they didn't work at both `CODEGEN` and `INTERPRETED` mode. Those test cases only work at `FALLBACK` mode. So, inevitably, we need to introduce `testFallback`.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31766 from dongjoon-hyun/SPARK-34596-SPARK-34607.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:41:49 -08:00
Cheng Su 14ad7afa1a [MINOR][SQL] Remove unnecessary extend from BroadcastHashJoinExec
### What changes were proposed in this pull request?

This is just a minor fix. `HashJoin` already extends `JoinCodegenSupport`. So we don't need `CodegenSupport` here for `BroadcastHashJoinExec`. Submitted separately as a PR here per https://github.com/apache/spark/pull/31802#discussion_r592066686 .

### Why are the changes needed?

Clean up code.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #31805 from c21/bhj-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:38:53 -08:00
Cheng Su 9aa8f06313 [SPARK-34711][SQL][TESTS] Exercise code-gen enable/disable code paths for SHJ in join test suites
### What changes were proposed in this pull request?

Per comment in https://github.com/apache/spark/pull/31802#discussion_r592068440 , we would like to exercise whole stage code-gen enabled and disabled code paths in join unit test suites. This is for better test coverage of shuffled hash join.

### Why are the changes needed?

Better test coverage.

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

No.

### How was this patch tested?

Existing and added unit tests here.

Closes #31806 from c21/test-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:34:09 -08:00
Anton Okolnychyi 72263797bc [SPARK-34457][SQL] DataSource V2: Add default null ordering to SortDirection
### What changes were proposed in this pull request?

This PR adds a default null ordering to public `SortDirection` to match the Catalyst behavior.

### Why are the changes needed?

The SQL standard does not define the default null ordering for a sort direction. That's why it is up to a query engine to assign one. We need to standardize this in our public connector expressions to avoid ambiguity. That's why I propose to match the behavior in our Catalyst expressions.

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

Yes, it affects unreleased connector expression API.

### How was this patch tested?

Existing tests.

Closes #31580 from aokolnychyi/spark-34457.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 05:47:31 +00:00
Terry Kim 2a6e68e1f7 [SPARK-34546][SQL] AlterViewAs.query should be analyzed during the analysis phase, and AlterViewAs should invalidate the cache
### What changes were proposed in this pull request?

This PR proposes the following:
   * `AlterViewAs.query` is currently analyzed in the physical operator `AlterViewAsCommand`, but it should be analyzed during the analysis phase.
   *  When `spark.sql.legacy.storeAnalyzedPlanForView` is set to true, store `TermporaryViewRelation` which wraps the analyzed plan, similar to #31273.
   *  Try to uncache the view you are altering.

### Why are the changes needed?

Analyzing a plan should be done in the analysis phase if possible.

Not uncaching the view (existing behavior) seems like a bug since the cache may not be used again.

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

Yes, now the view can be uncached if it's already cached.

### How was this patch tested?

Added new tests around uncaching.

The existing tests such as `SQLViewSuite` should cover the analysis changes.

Closes #31652 from imback82/alter_view_child.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 05:31:40 +00:00
Andy Grove fc182f7e7f [SPARK-34682][SQL] Use PrivateMethodTester instead of reflection
### Why are the changes needed?
SPARK-34682 was merged prematurely. This PR implements feedback from the review. I wasn't sure whether I should create a new JIRA or not.

### Does this PR introduce _any_ user-facing change?
No. Just improves the test.

### How was this patch tested?
Updated test.

Closes #31798 from andygrove/SPARK-34682-follow-up.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 12:08:31 -08:00
Andy Grove fd4843803c [SPARK-34682][SQL] Fix regression in canonicalization error check in CustomShuffleReaderExec
### What changes were proposed in this pull request?
There is a regression in 3.1.1 compared to 3.0.2 when checking for a canonicalized plan when executing CustomShuffleReaderExec.

The regression was caused by the call to `sendDriverMetrics` which happens before the check and will always fail if the plan is canonicalized.

### Why are the changes needed?
This is a regression in a useful error check.

### Does this PR introduce _any_ user-facing change?
No. This is not an error that a user would typically see, as far as I know.

### How was this patch tested?
I tested this change locally by making a distribution from this PR branch. Before fixing the regression I saw:

```
java.util.NoSuchElementException: key not found: numPartitions
```

After fixing this regression I saw:

```
java.lang.IllegalStateException: operating on canonicalized plan
```

Closes #31793 from andygrove/SPARK-34682.

Lead-authored-by: Andy Grove <andygrove73@gmail.com>
Co-authored-by: Andy Grove <andygrove@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-10 20:48:00 +09:00
Cheng Su a916690dd9 [SPARK-34681][SQL] Fix bug for full outer shuffled hash join when building left side with non-equal condition
### What changes were proposed in this pull request?

For full outer shuffled hash join with building hash map on left side, and having non-equal condition, the join can produce wrong result.

The root cause is `boundCondition` in `HashJoin.scala` always assumes the left side row is `streamedPlan` and right side row is `buildPlan` ([streamedPlan.output ++ buildPlan.output](https://github.com/apache/spark/blob/branch-3.1/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L141)). This is valid assumption, except for full outer + build left case.

The fix is to correct `boundCondition` in `HashJoin.scala` to handle full outer + build left case properly. See reproduce in https://issues.apache.org/jira/browse/SPARK-32399?focusedCommentId=17298414&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17298414 .

### Why are the changes needed?

Fix data correctness bug.

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

No.

### How was this patch tested?

Changed the test in `OuterJoinSuite.scala` to cover full outer shuffled hash join.
Before this change, the unit test `basic full outer join using ShuffledHashJoin` in `OuterJoinSuite.scala` is failed.

Closes #31792 from c21/join-bugfix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-09 22:55:27 -08:00
Wenchen Fan 48377d5bd9 [SPARK-34676][SQL][TEST] TableCapabilityCheckSuite should not inherit all tests from AnalysisSuite
### What changes were proposed in this pull request?

Fixes a mistake in `TableCapabilityCheckSuite`, which runs some tests repeatedly.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

N/A

Closes #31788 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-09 09:02:31 -08:00
Kousuke Saruta 2fd85174e9 [SPARK-34603][SQL] Support ADD ARCHIVE and LIST ARCHIVES command
### What changes were proposed in this pull request?

This PR adds `ADD ARCHIVE` and `LIST ARCHIVES` commands to SQL and updates relevant documents.
SPARK-33530 added `addArchive` and `listArchives` to `SparkContext` but it's not supported yet to add/list archives with SQL.

### Why are the changes needed?

To complement features.

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

No.

### How was this patch tested?

Added new test and confirmed the generated HTML from the updated documents.

Closes #31721 from sarutak/sql-archive.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-09 21:28:35 +09:00
Terry Kim cf8bbc184b [SPARK-34152][SQL][FOLLOW-UP] Global temp view's identifier should be correctly stored
### What changes were proposed in this pull request?

This PR proposed to fix a bug introduced in #31273 (https://github.com/apache/spark/pull/31273/files#r589494855).
### Why are the changes needed?

This fixes a bug where global temp view's database name was not passed correctly.

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

Yes, now the global temp view's database is correctly stored.

### How was this patch tested?

Added a new test that catches the bug.

Closes #31783 from imback82/SPARK-34152-bug-fix.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 12:08:08 +00:00
Amandeep Sharma a9c11896a5 [SPARK-34649][SQL][DOCS] org.apache.spark.sql.DataFrameNaFunctions.replace() fails for column name having a dot
### What changes were proposed in this pull request?

Use resolved attributes instead of data-frame fields for replacing values.

### Why are the changes needed?

dataframe.na.replace() does not work for column having a dot in the name

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

None

### How was this patch tested?

Added unit tests for the same

Closes #31769 from amandeep-sharma/master.

Authored-by: Amandeep Sharma <happyaman91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 11:47:01 +00:00
Cheng Su b5b198516c [SPARK-34620][SQL] Code-gen broadcast nested loop join (inner/cross)
### What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` does not have code-gen, and we can potentially boost the CPU performance for this operator if we add code-gen for it. https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html also showed the evidence in one fork.

The codegen for `BroadcastNestedLoopJoinExec` shared some code with `HashJoin`, and the interface `JoinCodegenSupport` is created to hold those common logic. This PR is only supporting inner and cross join. Other join types will be added later in followup PRs.

Example query and generated code:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 =!= $"k2").explain("codegen")
```

```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, Inner, NOT ((k1#2L + 1) = k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(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 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 038 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 039 */
/* 040 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 041 */
/* 042 */       long bnlj_value_4 = -1L;
/* 043 */
/* 044 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 045 */
/* 046 */       boolean bnlj_value_3 = false;
/* 047 */       bnlj_value_3 = bnlj_value_4 == bnlj_value_1;
/* 048 */       boolean bnlj_value_2 = false;
/* 049 */       bnlj_value_2 = !(bnlj_value_3);
/* 050 */       if (!(false || !bnlj_value_2))
/* 051 */       {
/* 052 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 053 */
/* 054 */         range_mutableStateArray_0[3].reset();
/* 055 */
/* 056 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 057 */
/* 058 */         range_mutableStateArray_0[3].write(1, bnlj_value_1);
/* 059 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 060 */
/* 061 */       }
/* 062 */     }
/* 063 */
/* 064 */   }
/* 065 */
/* 066 */   private void initRange(int idx) {
/* 067 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 068 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 069 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 070 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 071 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 072 */     long partitionEnd;
/* 073 */
/* 074 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 075 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 076 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 077 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 078 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 079 */     } else {
/* 080 */       range_nextIndex_0 = st.longValue();
/* 081 */     }
/* 082 */     range_batchEnd_0 = range_nextIndex_0;
/* 083 */
/* 084 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 085 */     .multiply(step).add(start);
/* 086 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 087 */       partitionEnd = Long.MAX_VALUE;
/* 088 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 089 */       partitionEnd = Long.MIN_VALUE;
/* 090 */     } else {
/* 091 */       partitionEnd = end.longValue();
/* 092 */     }
/* 093 */
/* 094 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 095 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 096 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 097 */     if (range_numElementsTodo_0 < 0) {
/* 098 */       range_numElementsTodo_0 = 0;
/* 099 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 100 */       range_numElementsTodo_0++;
/* 101 */     }
/* 102 */   }
/* 103 */
/* 104 */   protected void processNext() throws java.io.IOException {
/* 105 */     // initialize Range
/* 106 */     if (!range_initRange_0) {
/* 107 */       range_initRange_0 = true;
/* 108 */       initRange(partitionIndex);
/* 109 */     }
/* 110 */
/* 111 */     while (true) {
/* 112 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 113 */         long range_nextBatchTodo_0;
/* 114 */         if (range_numElementsTodo_0 > 1000L) {
/* 115 */           range_nextBatchTodo_0 = 1000L;
/* 116 */           range_numElementsTodo_0 -= 1000L;
/* 117 */         } else {
/* 118 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 119 */           range_numElementsTodo_0 = 0;
/* 120 */           if (range_nextBatchTodo_0 == 0) break;
/* 121 */         }
/* 122 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 123 */       }
/* 124 */
/* 125 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 126 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 127 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 128 */
/* 129 */         // common sub-expressions
/* 130 */
/* 131 */         bnlj_doConsume_0(range_value_0);
/* 132 */
/* 133 */         if (shouldStop()) {
/* 134 */           range_nextIndex_0 = range_value_0 + 1L;
/* 135 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 136 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 137 */           return;
/* 138 */         }
/* 139 */
/* 140 */       }
/* 141 */       range_nextIndex_0 = range_batchEnd_0;
/* 142 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 143 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 144 */       range_taskContext_0.killTaskIfInterrupted();
/* 145 */     }
/* 146 */   }
/* 147 */
/* 148 */ }
```

### Why are the changes needed?

Improve query CPU performance. Added a micro benchmark query in `JoinBenchmark.scala`.
Saw 1x of run time improvement:

```
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------
broadcast nested loop join wholestage off          62922          63052         184          0.3        3000.3       1.0X
broadcast nested loop join wholestage on           30946          30972          26          0.7        1475.6       2.0X
```

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

No.

### How was this patch tested?

* Added unit test in `WholeStageCodegenSuite.scala`, and existing unit tests for `BroadcastNestedLoopJoinExec`.
* Updated golden files for several TCPDS query plans, as whole stage code-gen for `BroadcastNestedLoopJoinExec` is triggered.
* Updated `JoinBenchmark-jdk11-results.txt ` and `JoinBenchmark-results.txt` with new benchmark result. Followed previous benchmark PRs - https://github.com/apache/spark/pull/27078 and https://github.com/apache/spark/pull/26003 to use same type of machine:

```
Amazon AWS EC2
type: r3.xlarge
region: us-west-2 (Oregon)
OS: Linux
```

Closes #31736 from c21/nested-join-exec.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 11:45:43 +00:00
Takeshi Yamamuro 43b23fd132 [SPARK-33498][SQL][TESTS][FOLLOWUP] Remove SQLConf.withExistingConf in CastSuite
### What changes were proposed in this pull request?

This PR intends to remove unnecessary `SQLConf.withExistingConf` in `CastSuite`; since we've remove `ParVector ` in #31775, we no longer need to copy SQL configs into each thread env.

### Why are the changes needed?

Clean up the code.

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

No.

### How was this patch tested?

Run the existing tests.

Closes #31785 from maropu/UpdateCastSuite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-08 23:52:43 -08:00
Max Gekk 6a7701a5be [SPARK-34663][SQL][TESTS] Test year-month and day-time intervals in UDF
### What changes were proposed in this pull request?
Added new tests to `UDFSuite` to check `java.time.Period`/`java.time.Duration` in UDF as input parameters as well as UDF results.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *UDFSuite"
```

Closes #31779 from MaxGekk/interval-udf.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 06:58:14 +00:00
Max Gekk 4ea27787bf [SPARK-34666][SQL][TESTS] Test DayTimeIntervalType and YearMonthIntervalType as ordered and atomic types
### What changes were proposed in this pull request?
Add `DayTimeIntervalType` and `YearMonthIntervalType` to `DataTypeTestUtils.ordered`/`atomicTypes`, and implement values generation of those types in `LiteralGenerator`/`RandomDataGenerator`. In this way, the types will be tested automatically in:
1. ArithmeticExpressionSuite:
    - "function least"
    - "function greatest"
2. PredicateSuite
    - "BinaryComparison consistency check"
    - "AND, OR, EqualTo, EqualNullSafe consistency check"
3. ConditionalExpressionSuite
    - "if"
4. RandomDataGeneratorSuite
    - "Basic types"
5. CastSuite
    - "null cast"
    - "up-cast"
    - "SPARK-27671: cast from nested null type in struct"
6. OrderingSuite
    - "GenerateOrdering with DayTimeIntervalType"
    - "GenerateOrdering with YearMonthIntervalType"
7. PredicateSuite
    - "IN with different types"
8. UnsafeRowSuite
    - "calling get(ordinal, datatype) on null columns"
9. SortSuite
    - "sorting on YearMonthIntervalType ..."
    - "sorting on DayTimeIntervalType ..."

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running the affected test suites.

Closes #31782 from MaxGekk/test-interval-as-atomic.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 06:42:59 +00:00
allisonwang-db c32cac4cd6 [SPARK-34627][SQL] Use FunctionIdentifier in UnresolvedTableValuedFunction
### What changes were proposed in this pull request?
This PR updates UnresolvedTableValuedFunction's name to be a FunctionIdentifier instead of a string.

### Why are the changes needed?
To make UnresolvedTableValuedFunction consistent with UnresolvedFunction that uses FunctionIdentifier as the function name.

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

### How was this patch tested?
Unit test.

Closes #31749 from allisonwang-db/spark-34627.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 05:27:02 +00:00
Swinky 02e74b298a [SPARK-34598][SQL] RewritePredicateSubquery Rule must not update Filters without subqueries
### What changes were proposed in this pull request?
RewritePredicateSubquery Optimizer Rule must not update Filters without subqueries.

Following is one such example.

```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery ===
 Project [a#0]                                                        Project [a#0]
!+- Filter (((a#0 > 1) OR (b#1 > 2)) AND ((c#2 > 1) AND (d#3 > 2)))   +- Filter ((((a#0 > 1) OR (b#1 > 2)) AND (c#2 > 1)) AND (d#3 > 2))
    +- LocalRelation <empty>, [a#0, b#1, c#2, d#3]                       +- LocalRelation <empty>, [a#0, b#1, c#2, d#3]
```

### Why are the changes needed?
minor change.

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

### How was this patch tested?
Existing UTs pass.

Closes #31712 from Swinky/rewritePredicateFix.

Authored-by: Swinky <mannswinky@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-08 09:01:49 +00:00
Max Gekk e10bf64769 [SPARK-34615][SQL] Support java.time.Period as an external type of the year-month interval type
### What changes were proposed in this pull request?
In the PR, I propose to extend Spark SQL API to accept [`java.time.Period`](https://docs.oracle.com/javase/8/docs/api/java/time/Period.html) as an external type of recently added new Catalyst type - `YearMonthIntervalType` (see #31614). The Java class `java.time.Period` has similar semantic to ANSI SQL year-month interval type, and it is the most suitable to be an external type for `YearMonthIntervalType`. In more details:
1. Added `PeriodConverter` which converts `java.time.Period` instances to/from internal representation of the Catalyst type `YearMonthIntervalType` (to `Int` type). The `PeriodConverter` object uses new methods of `IntervalUtils`:
    - `periodToMonths()` converts the input period to the total length in months. If this period is too large to fit `Int`, the method throws the exception `ArithmeticException`. **Note:** _the input period has "days" precision, the method just ignores the days unit._
    - `monthToPeriod()` obtains a `java.time.Period` representing a number of months.
2. Support new type `YearMonthIntervalType` in `RowEncoder` via the methods `createDeserializerForPeriod()` and `createSerializerForJavaPeriod()`.
3. Extended the Literal API to construct literals from `java.time.Period` instances.

### Why are the changes needed?
1. To allow users parallelization of `java.time.Period` collections, and construct year-month interval columns. Also to collect such columns back to the driver side.
2. This will allow to write tests in other sub-tasks of SPARK-27790.

### Does this PR introduce _any_ user-facing change?
The PR extends existing functionality. So, users can parallelize instances of the `java.time.Duration` class and collect them back:

```scala
scala> val ds = Seq(java.time.Period.ofYears(10).withMonths(2)).toDS
ds: org.apache.spark.sql.Dataset[java.time.Period] = [value: yearmonthinterval]

scala> ds.collect
res0: Array[java.time.Period] = Array(P10Y2M)
```

### How was this patch tested?
- Added a few tests to `CatalystTypeConvertersSuite` to check conversion from/to `java.time.Period`.
- Checking row encoding by new tests in `RowEncoderSuite`.
- Making literals of `YearMonthIntervalType` are tested in `LiteralExpressionSuite`.
- Check collecting by `DatasetSuite` and `JavaDatasetSuite`.
- New tests in `IntervalUtilsSuites` to check conversions `java.time.Period` <-> months.

Closes #31765 from MaxGekk/java-time-period.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-08 08:33:09 +00:00
yangjie01 43f355b5f2 [SPARK-34597][SQL] Replaces ParquetFileReader.readFooter with ParquetFileReader.open and getFooter
### What changes were proposed in this pull request?
`ParquetFileReader.readFooter` related methods has been identified as `Deprecated` and `Apache Parquet` suggests replace it with the combination of `ParquetFileReader.open() and getFooter()` methods.

This PR introduces the `ParquetFooterReader` utility class due to some repetitive code patterns when read parquet file footer.

### 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 #31711 from LuciferYang/parquet-read-footer.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-07 23:38:40 -08:00
Peter Toth ab8a9a0ceb [SPARK-34545][SQL] Fix issues with valueCompare feature of pyrolite
### What changes were proposed in this pull request?

pyrolite 4.21 introduced and enabled value comparison by default (`valueCompare=true`) during object memoization and serialization: https://github.com/irmen/Pyrolite/blob/pyrolite-4.21/java/src/main/java/net/razorvine/pickle/Pickler.java#L112-L122
This change has undesired effect when we serialize a row (actually `GenericRowWithSchema`) to be passed to python: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L60. A simple example is that
```
new GenericRowWithSchema(Array(1.0, 1.0), StructType(Seq(StructField("_1", DoubleType), StructField("_2", DoubleType))))
```
and
```
new GenericRowWithSchema(Array(1, 1), StructType(Seq(StructField("_1", IntegerType), StructField("_2", IntegerType))))
```
are currently equal and the second instance is replaced to the short code of the first one during serialization.

### Why are the changes needed?
The above can cause nasty issues like the one in https://issues.apache.org/jira/browse/SPARK-34545 description:

```
>>> from pyspark.sql.functions import udf
>>> from pyspark.sql.types import *
>>>
>>> def udf1(data_type):
        def u1(e):
            return e[0]
        return udf(u1, data_type)
>>>
>>> df = spark.createDataFrame([((1.0, 1.0), (1, 1))], ['c1', 'c2'])
>>>
>>> df = df.withColumn("c3", udf1(DoubleType())("c1"))
>>> df = df.withColumn("c4", udf1(IntegerType())("c2"))
>>>
>>> df.select("c3").show()
+---+
| c3|
+---+
|1.0|
+---+

>>> df.select("c4").show()
+---+
| c4|
+---+
|  1|
+---+

>>> df.select("c3", "c4").show()
+---+----+
| c3|  c4|
+---+----+
|1.0|null|
+---+----+
```
This is because during serialization from JVM to Python `GenericRowWithSchema(1.0, 1.0)` (`c1`) is memoized first and when `GenericRowWithSchema(1, 1)` (`c2`) comes next, it is replaced to some short code of the `c1` (instead of serializing `c2` out) as they are `equal()`. The python functions then runs but the return type of `c4` is expected to be `IntegerType` and if a different type (`DoubleType`) comes back from python then it is discarded: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L108-L113

After this PR:
```
>>> df.select("c3", "c4").show()
+---+---+
| c3| c4|
+---+---+
|1.0|  1|
+---+---+
```

### Does this PR introduce _any_ user-facing change?
Yes, fixes a correctness issue.

### How was this patch tested?
Added new UT + manual tests.

Closes #31682 from peter-toth/SPARK-34545-fix-row-comparison.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-07 19:12:42 -06:00
wangguangxin.cn 9ec8696f11 [SPARK-34634][SQL] ResolveReferences.dedupRight should handle ScriptTransformation
### What changes were proposed in this pull request?
When we do self join with transform in a CTE, spark will throw AnalysisException.

A simple way to reproduce is

```
create temporary view t as select * from values 0, 1, 2 as t(a);

WITH temp AS (
  SELECT TRANSFORM(a) USING 'cat' AS (b string) FROM t
)
SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b
```

before this patch, it throws

```
org.apache.spark.sql.AnalysisException: cannot resolve '`t1.b`' given input columns: [t1.b]; line 6 pos 41;
'Project ['t1.b]
+- 'Join Inner, ('t1.b = 't2.b)
   :- SubqueryAlias t1
   :  +- SubqueryAlias temp
   :     +- ScriptTransformation [a#1], cat, [b#2], ScriptInputOutputSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.DelimitedJSONSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,	)),List((field.delim,	)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
   :        +- SubqueryAlias t
   :           +- Project [a#1]
   :              +- SubqueryAlias t
   :                 +- LocalRelation [a#1]
   +- SubqueryAlias t2
      +- SubqueryAlias temp
         +- ScriptTransformation [a#1], cat, [b#2], ScriptInputOutputSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.DelimitedJSONSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,	)),List((field.delim,	)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
            +- SubqueryAlias t
               +- Project [a#1]
                  +- SubqueryAlias t
                     +- LocalRelation [a#1]
```

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

### How was this patch tested?
Add a UT

Closes #31752 from WangGuangxin/selfjoin-with-transform.

Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-07 15:53:52 +09:00
Yuming Wang 616c818e7c [SPARK-34628][SQL] Remove GlobalLimit operator if its child max rows not larger than limit number
### What changes were proposed in this pull request?

This pr remove `GlobalLimit` operator if its child max rows not larger than limit number. For example:
```
val testRelation = LocalRelation.fromExternalRows(Seq("a".attr.int, "b".attr.int, "c".attr.int), 1.to(10).map(_ => Row(1, 2, 3)) )
val query = GlobalLimit(100, testRelation)
```
We can remove this `GlobalLimit`.

### Why are the changes needed?

Further optimize the query.

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

No.

### How was this patch tested?

Unit test.

Closes #31750 from wangyum/SPARK-34628.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-06 09:54:15 -08:00
helloman 1a9722420e [SPARK-34595][SQL] DPP support RLIKE
### What changes were proposed in this pull request?
This pr make DPP support RLIKE expression:

```sql
SELECT date_id, product_id FROM fact_sk f
JOIN dim_store s
ON f.store_id = s.store_id WHERE s.country RLIKE  '[DE|US]'
```
 ### Why are the changes needed?
Improve query performance.

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

 ### How was this patch tested?
 Unit test.

Closes #31722 from chaojun-zhang/SPARK-34595.

Authored-by: helloman <zcj23085@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-06 20:30:20 +09:00
Angerszhuuuu 654f19dfd7 [SPARK-34621][SQL] Unify output of ShowCreateTableAsSerdeCommand and ShowCreateTableCommand
### What changes were proposed in this pull request?
Unify output of ShowCreateTableAsSerdeCommand  and ShowCreateTableCommand

### Why are the changes needed?
Unify output of ShowCreateTableAsSerdeCommand  and ShowCreateTableCommand

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

### How was this patch tested?

Closes #31737 from AngersZhuuuu/SPARK-34621.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 07:44:07 +00:00
suqilong ca326c4bb3 [SPARK-22748][SQL] Analyze __grouping__id as a literal function
### What changes were proposed in this pull request?

This PR intends to refactor the logic to resolve `__grouping_id` in the `Analyzer`; it moves the logic from `ResolveFunctions` to `ResolveReferences` (`resolveLiteralFunction`).

The original author of this PR is sqlwindspeaker (#30781).

Closes #30781.

### Why are the changes needed?

Code refactoring.

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

No.

### How was this patch tested?

Added tests in `AnalysisSuite`.

Closes #31751 from maropu/SPARK-22748.

Authored-by: suqilong <suqilong@qiyi.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 07:40:58 +00:00
Wenchen Fan dc78f337cb [SPARK-34609][SQL] Unify resolveExpressionBottomUp and resolveExpressionTopDown
### What changes were proposed in this pull request?

It's a bit confusing to see `resolveExpressionBottomUp` and `resolveExpressionTopDown`, which provide similar functionalities but with different tree traverse order. It turns out that the real difference between these 2 methods is: which attributes should the columns be resolved to? `resolveExpressionTopDown` resolves columns using output attributes of the plan children, `resolveExpressionBottomUp` resolves columns using output attributes of the plan itself.

This PR unifies `resolveExpressionBottomUp` and `resolveExpressionTopDown` and put the common logic in a new method, and let `resolveExpressionBottomUp` and `resolveExpressionTopDown` just call the new method. This PR also renames `resolveExpressionBottomUp` and `resolveExpressionTopDown` to make the difference clear.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31728 from cloud-fan/resolve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 05:59:15 +00:00
Angerszhuuuu 979b9bcf5d [SPARK-34608][SQL] Remove unused output of AddJarCommand
### What changes were proposed in this pull request?
Remove unused output of AddJarCommand, keep consistence and clean

### Why are the changes needed?
Keep consistence and clean

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

### How was this patch tested?
Not need

Closes #31725 from AngersZhuuuu/SPARK-34608.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-04 21:19:07 -08:00
ulysses-you 43aacd5069 [SPARK-34613][SQL] Fix view does not capture disable hint config
### What changes were proposed in this pull request?

Add allow list to capture sql config for view.

### Why are the changes needed?

Spark use origin text sql to store view then capture and store sql config into view metadata.

Capture config will skip some config with some prefix, e.g. `spark.sql.optimizer.` but unfortunately `spark.sql.optimizer.disableHints` is start with `spark.sql.optimizer.`.

We need a allow list to help capture the config.

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

Yes bug fix.

### How was this patch tested?

Add test.

Closes #31732 from ulysses-you/SPARK-34613.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 12:19:30 +08:00
Kent Yao 814d81c1e5 [SPARK-34376][SQL] Support regexp as a SQL function
### What changes were proposed in this pull request?

We have equality in `SqlBase.g4` for `RLIKE: 'RLIKE' | 'REGEXP';`
We seemed to miss adding` REGEXP` as a SQL function just like` RLIKE`

### Why are the changes needed?

symmetry and beauty
This is also a builtin  function in Hive, we can reduce the migration pain for those users

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

yes new regexp function as an alias as rlike

### How was this patch tested?

new tests

Closes #31488 from yaooqinn/SPARK-34376.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-05 12:09:28 +09:00
Takeshi Yamamuro dbce74d39d [SPARK-34607][SQL] Add Utils.isMemberClass to fix a malformed class name error on jdk8u
### What changes were proposed in this pull request?

This PR intends to fix a bug of `objects.NewInstance` if a user runs Spark on jdk8u and a given `cls` in `NewInstance` is a deeply-nested inner class, e.g.,.
```
  object OuterLevelWithVeryVeryVeryLongClassName1 {
    object OuterLevelWithVeryVeryVeryLongClassName2 {
      object OuterLevelWithVeryVeryVeryLongClassName3 {
        object OuterLevelWithVeryVeryVeryLongClassName4 {
          object OuterLevelWithVeryVeryVeryLongClassName5 {
            object OuterLevelWithVeryVeryVeryLongClassName6 {
              object OuterLevelWithVeryVeryVeryLongClassName7 {
                object OuterLevelWithVeryVeryVeryLongClassName8 {
                  object OuterLevelWithVeryVeryVeryLongClassName9 {
                    object OuterLevelWithVeryVeryVeryLongClassName10 {
                      object OuterLevelWithVeryVeryVeryLongClassName11 {
                        object OuterLevelWithVeryVeryVeryLongClassName12 {
                          object OuterLevelWithVeryVeryVeryLongClassName13 {
                            object OuterLevelWithVeryVeryVeryLongClassName14 {
                              object OuterLevelWithVeryVeryVeryLongClassName15 {
                                object OuterLevelWithVeryVeryVeryLongClassName16 {
                                  object OuterLevelWithVeryVeryVeryLongClassName17 {
                                    object OuterLevelWithVeryVeryVeryLongClassName18 {
                                      object OuterLevelWithVeryVeryVeryLongClassName19 {
                                        object OuterLevelWithVeryVeryVeryLongClassName20 {
                                          case class MalformedNameExample2(x: Int)
                                        }}}}}}}}}}}}}}}}}}}}
```

The root cause that Kris (rednaxelafx) investigated is as follows (Kudos to Kris);

The reason why the test case above is so convoluted is in the way Scala generates the class name for nested classes. In general, Scala generates a class name for a nested class by inserting the dollar-sign ( `$` ) in between each level of class nesting. The problem is that this format can concatenate into a very long string that goes beyond certain limits, so Scala will change the class name format beyond certain length threshold.

For the example above, we can see that the first two levels of class nesting have class names that look like this:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$OuterLevelWithVeryVeryVeryLongClassName2$
```
If we leave out the fact that Scala uses a dollar-sign ( `$` ) suffix for the class name of the companion object, `OuterLevelWithVeryVeryVeryLongClassName1`'s full name is a prefix (substring) of `OuterLevelWithVeryVeryVeryLongClassName2`.

But if we keep going deeper into the levels of nesting, you'll find names that look like:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$OuterLevelWithVeryVeryVeryLongClassName11$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$OuterLevelWithVeryVeryVeryLongClassName14$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$OuterLevelWithVeryVeryVeryLongClassName17$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$OuterLevelWithVeryVeryVeryLongClassName20$
```
with a hash code in the middle and various levels of nesting omitted.

The `java.lang.Class.isMemberClass` method is implemented in JDK8u as:
http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/tip/src/share/classes/java/lang/Class.java#l1425
```
    /**
     * Returns {code true} if and only if the underlying class
     * is a member class.
     *
     * return {code true} if and only if this class is a member class.
     * since 1.5
     */
    public boolean isMemberClass() {
        return getSimpleBinaryName() != null && !isLocalOrAnonymousClass();
    }

    /**
     * Returns the "simple binary name" of the underlying class, i.e.,
     * the binary name without the leading enclosing class name.
     * Returns {code null} if the underlying class is a top level
     * class.
     */
    private String getSimpleBinaryName() {
        Class<?> enclosingClass = getEnclosingClass();
        if (enclosingClass == null) // top level class
            return null;
        // Otherwise, strip the enclosing class' name
        try {
            return getName().substring(enclosingClass.getName().length());
        } catch (IndexOutOfBoundsException ex) {
            throw new InternalError("Malformed class name", ex);
        }
    }
```
and the problematic code is `getName().substring(enclosingClass.getName().length())` -- if a class's enclosing class's full name is *longer* than the nested class's full name, this logic would end up going out of bounds.

The bug has been fixed in JDK9 by https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8057919 , but still exists in the latest JDK8u release. So from the Spark side we'd need to do something to avoid hitting this problem.

### Why are the changes needed?

Bugfix on jdk8u.

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

No.

### How was this patch tested?

Added tests.

Closes #31733 from maropu/SPARK-34607.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-05 08:59:30 +09:00
Max Gekk 17601e014c [SPARK-34605][SQL] Support java.time.Duration as an external type of the day-time interval type
### What changes were proposed in this pull request?
In the PR, I propose to extend Spark SQL API to accept [`java.time.Duration`](https://docs.oracle.com/javase/8/docs/api/java/time/Duration.html) as an external type of recently added new Catalyst type - `DayTimeIntervalType` (see #31614). The Java class `java.time.Duration` has similar semantic to ANSI SQL day-time interval type, and it is the most suitable to be an external type for `DayTimeIntervalType`. In more details:
1. Added `DurationConverter` which converts `java.time.Duration` instances to/from internal representation of the Catalyst type `DayTimeIntervalType` (to `Long` type). The `DurationConverter` object uses new methods of `IntervalUtils`:
    - `durationToMicros()` converts the input duration to the total length in microseconds. If this duration is too large to fit `Long`, the method throws the exception `ArithmeticException`. **Note:** _the input duration has nanosecond precision, the method casts the nanos part to microseconds by dividing by 1000._
    - `microsToDuration()` obtains a `java.time.Duration` representing a number of microseconds.
2. Support new type `DayTimeIntervalType` in `RowEncoder` via the methods `createDeserializerForDuration()` and `createSerializerForJavaDuration()`.
3. Extended the Literal API to construct literals from `java.time.Duration` instances.

### Why are the changes needed?
1. To allow users parallelization of `java.time.Duration` collections, and construct day-time interval columns. Also to collect such columns back to the driver side.
2. This will allow to write tests in other sub-tasks of SPARK-27790.

### Does this PR introduce _any_ user-facing change?
The PR extends existing functionality. So, users can parallelize instances of the `java.time.Duration` class and collect them back:

```Scala
scala> val ds = Seq(java.time.Duration.ofDays(10)).toDS
ds: org.apache.spark.sql.Dataset[java.time.Duration] = [value: daytimeinterval]

scala> ds.collect
res0: Array[java.time.Duration] = Array(PT240H)
```

### How was this patch tested?
- Added a few tests to `CatalystTypeConvertersSuite` to check conversion from/to `java.time.Duration`.
- Checking row encoding by new tests in `RowEncoderSuite`.
- Making literals of `DayTimeIntervalType` are tested in `LiteralExpressionSuite`
- Check collecting by `DatasetSuite` and `JavaDatasetSuite`.

Closes #31729 from MaxGekk/java-time-duration.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 16:58:33 +00:00
yi.wu e7e016192f [SPARK-34482][SS] Correct the active SparkSession for StreamExecution.logicalPlan
### What changes were proposed in this pull request?

Set the active SparkSession to `sparkSessionForStream` and diable AQE & CBO before initializing the `StreamExecution.logicalPlan`.

### Why are the changes needed?

The active session should be `sparkSessionForStream`. Otherwise, settings like

6b34745cb9/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala (L332-L335)

wouldn't take effect if callers access them from the active SQLConf, e.g., the rule of `InsertAdaptiveSparkPlan`. Besides, unlike `InsertAdaptiveSparkPlan` (which skips streaming plan), `CostBasedJoinReorder` seems to have the chance to take effect theoretically.

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

No.

### How was this patch tested?

Tested manually. Before the fix, `InsertAdaptiveSparkPlan` would try to apply AQE on the plan(wouldn't take effect though). After this fix, the rule returns directly.

Closes #31600 from Ngone51/active-session-for-stream.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 22:41:11 +08:00
Angerszhuuuu 401e270c17 [SPARK-34567][SQL] CreateTableAsSelect should update metrics too
### What changes were proposed in this pull request?
For command `CreateTableAsSelect` we use `InsertIntoHiveTable`, `InsertIntoHadoopFsRelationCommand` to insert data.
We will update metrics of  `InsertIntoHiveTable`, `InsertIntoHadoopFsRelationCommand`  in `FileFormatWriter.write()`, but we only show CreateTableAsSelectCommand in WebUI SQL Tab.
We need to update `CreateTableAsSelectCommand`'s metrics too.

Before this PR:
![image](https://user-images.githubusercontent.com/46485123/109411226-81f44480-79db-11eb-99cb-b9686b15bf61.png)

After this PR:
![image](https://user-images.githubusercontent.com/46485123/109411232-8ae51600-79db-11eb-9111-3bea0bc2d475.png)

![image](https://user-images.githubusercontent.com/46485123/109905192-62aa2f80-7cd9-11eb-91f9-04b16c9238ae.png)

### Why are the changes needed?
Complete SQL Metrics

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

### How was this patch tested?
<!--
MT

Closes #31679 from AngersZhuuuu/SPARK-34567.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 20:42:47 +08:00
Gengliang Wang 2b1c170016 [SPARK-34614][SQL] ANSI mode: Casting String to Boolean should throw exception on parse error
### What changes were proposed in this pull request?

In ANSI mode, casting String to Boolean should throw an exception on parse error, instead of returning null

### Why are the changes needed?

For better ANSI compliance

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

Yes, in ANSI mode there will be an exception on parse failure of casting String value to Boolean type.

### How was this patch tested?

Unit tests.

Closes #31734 from gengliangwang/ansiCastToBoolean.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2021-03-04 19:04:16 +08:00
Shixiong Zhu 53e4dba7c4 [SPARK-34599][SQL] Fix the issue that INSERT INTO OVERWRITE doesn't support partition columns containing dot for DSv2
### What changes were proposed in this pull request?

`ResolveInsertInto.staticDeleteExpression` should use `UnresolvedAttribute.quoted` to create the delete expression so that we will treat the entire `attr.name` as a column name.

### Why are the changes needed?

When users use `dot` in a partition column name, queries like ```INSERT OVERWRITE $t1 PARTITION (`a.b` = 'a') (`c.d`) VALUES('b')``` is not working.

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

Without this test, the above query will throw
```
[info]   org.apache.spark.sql.AnalysisException: cannot resolve '`a.b`' given input columns: [a.b, c.d];
[info] 'OverwriteByExpression RelationV2[a.b#17, c.d#18] default.tbl, ('a.b <=> cast(a as string)), false
[info] +- Project [a.b#19, ansi_cast(col1#16 as string) AS c.d#20]
[info]    +- Project [cast(a as string) AS a.b#19, col1#16]
[info]       +- LocalRelation [col1#16]
```

With the fix, the query will run correctly.

### How was this patch tested?

The new added test.

Closes #31713 from zsxwing/SPARK-34599.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 15:12:53 +08:00
Angerszhuuuu db627107b7 [SPARK-34577][SQL] Fix drop/add columns to a dataset of DESCRIBE NAMESPACE
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the DESCRIBE NAMESPACE command.

### Why are the changes needed?
This fixes the issue demonstrated by the example:

```
sql(s"CREATE NAMESPACE ns")
val description = sql(s"DESCRIBE NAMESPACE ns")
description.drop("name")
```

```
[info]   org.apache.spark.sql.AnalysisException: Resolved attribute(s) name#74 missing from name#25,value#26 in operator !Project [name#74]. Attribute(s) with the same name appear in the operation: name. Please check if the right attribute(s) are used.;
[info] !Project [name#74]
[info] +- LocalRelation [name#25, value#26]
```

### Does this PR introduce _any_ user-facing change?
After this change user `drop()/add()` works well.

### How was this patch tested?
Added UT

Closes #31705 from AngersZhuuuu/SPARK-34577.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 13:22:10 +08:00
Wenchen Fan 8f1eec4d13 [SPARK-34584][SQL] Static partition should also follow StoreAssignmentPolicy when insert into v2 tables
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/27597 and simply apply the fix in the v2 table insertion code path.

### Why are the changes needed?

bug fix

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

yes, now v2 table insertion with static partitions also follow StoreAssignmentPolicy.

### How was this patch tested?

moved the test from https://github.com/apache/spark/pull/27597 to the general test suite `SQLInsertTestSuite`, which covers DS v2, file source, and hive tables.

Closes #31726 from cloud-fan/insert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-04 11:29:34 +09:00
Erik Krogen 9d2d620e98 [SPARK-33084][SQL][TEST][FOLLOWUP] Add ResetSystemProperties trait to SQLQuerySuite to avoid clearing ivy.home
### What changes were proposed in this pull request?
Add the `ResetSystemProperties` trait to `SQLQuerySuite` so that system property changes made by any of the tests will not affect other suites/tests. Specifically, the system property changes made by `SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class` are targeted here (which sets and then clears `ivy.home`).

### Why are the changes needed?
PR #29966 added a new test case that adjusts the `ivy.home` system property to force Ivy to resolve an artifact from a custom location. At the end of the test, the value is cleared. Clearing the value meant that, if a custom value of `ivy.home` was configured externally, it would not apply for tests run after this test case.

### Does this PR introduce _any_ user-facing change?
No, this is only in tests.

### How was this patch tested?
Existing unit tests continue to pass, whether or not `spark.jars.ivySettings` is configured (which adjusts the behavior of Ivy w.r.t. handling of `ivy.home` and `ivy.default.ivy.user.dir` properties).

Closes #31694 from xkrogen/xkrogen-SPARK-33084-ivyhome-sysprop-followon.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-04 08:52:53 +09:00
Gengliang Wang 5aaab19685 [SPARK-34222][SQL][FOLLOWUP] Non-recursive implementation of buildBalancedPredicate
### What changes were proposed in this pull request?

Use a non-recursive implementation for the function buildBalancedPredicate
### Why are the changes needed?

For better performance.

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

No
### How was this patch tested?

Existing unit tests.
Also, a quick benchmark:
```
  test("buildBalancedPredicate") {
    val expressions = (1 to 1000).map(_ => Literal(true))
    val start = System.currentTimeMillis()
    buildBalancedPredicate(expressions, And)
    println(System.currentTimeMillis() - start)
  }
```
Before: 47ms
After: 4ms

Closes #31724 from gengliangwang/nonrecursive.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2021-03-04 01:01:28 +08:00
Karen Feng b01dd12805 [SPARK-34555][SQL] Resolve metadata output from DataFrame
### What changes were proposed in this pull request?

Add metadataOutput as a fallback to resolution.
Builds off https://github.com/apache/spark/pull/31654.

### Why are the changes needed?

The metadata columns could not be resolved via `df.col("metadataColName")` from the DataFrame API.

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

Yes, the metadata columns can now be resolved as described above.

### How was this patch tested?

Scala unit test.

Closes #31668 from karenfeng/spark-34555.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 22:07:41 +08:00
angerszhu 56edb8156f [SPARK-33474][SQL] Support TypeConstructed partition spec value
### What changes were proposed in this pull request?
Hive support type constructed value as partition spec value, spark should support too.

### Why are the changes needed?
 Support TypeConstructed partition spec value keep same with hive

### Does this PR introduce _any_ user-facing change?
Yes, user can use TypeConstruct value as partition spec value such as
```
CREATE TABLE t1(name STRING) PARTITIONED BY (part DATE)
INSERT INTO t1 PARTITION(part = date'2019-01-02') VALUES('a')

CREATE TABLE t2(name STRING) PARTITIONED BY (part TIMESTAMP)
INSERT INTO t2 PARTITION(part = timestamp'2019-01-02 11:11:11') VALUES('a')

CREATE TABLE t4(name STRING) PARTITIONED BY (part BINARY)
INSERT INTO t4 PARTITION(part = X'537061726B2053514C') VALUES('a')
```

### How was this patch tested?
Added UT

Closes #30421 from AngersZhuuuu/SPARK-33474.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-03 16:48:50 +09:00
Kent Yao 499f620037 [MINOR][SQL][DOCS] Fix some wrong default values in SQL tuning guide's AQE section
### What changes were proposed in this pull request?

spark.sql.adaptive.coalescePartitions.initialPartitionNum 200 -> (none)
spark.sql.adaptive.skewJoin.skewedPartitionFactor is 10 -> 5

### Why are the changes needed?

the wrong doc misguide people
### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

passing doc

Closes #31717 from yaooqinn/minordoc0.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 15:00:09 +09:00
Swinky 229d2e0554 [SPARK-34222][SQL] Enhance boolean simplification rule
### What changes were proposed in this pull request?
Enhance boolean simplification rule by handling following scenarios:
(((a && b) && a && (a && c))) => a && b && c)
(((a || b) || a || (a || c))) => a || b || c

### Why are the changes needed?
Minor improvement

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

### How was this patch tested?
Added UTs

Closes #31318 from Swinky/booleansimplification.

Authored-by: Swinky <mannswinky@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 05:25:28 +00:00
Angerszhuuuu 17f0e70fa0 [SPARK-34576][SQL] Fix drop/add columns to a dataset of DESCRIBE COLUMN
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the DESCRIBE COLUMN command.

### Why are the changes needed?
This fixes the issue demonstrated by the example:

```
val tbl = "testcat.ns1.ns2.tbl"
sql(s"CREATE TABLE $tbl (c0 INT) USING _")
val description = sql(s"DESCRIBE TABLE $tbl c0")
description.drop("info_name")
```

```
[info]   org.apache.spark.sql.AnalysisException: Resolved attribute(s) info_name#74 missing from info_name#25,info_value#26 in operator !Project [info_name#74]. Attribute(s) with the same name appear in the operation: info_name. Please check if the right attribute(s) are used.;
[info] !Project [info_name#74]
[info] +- LocalRelation [info_name#25, info_value#26]
```

### Does this PR introduce _any_ user-facing change?
After this change user `drop()/add()` works well.

### How was this patch tested?
Added UT

Closes #31696 from AngersZhuuuu/SPARK-34576.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 12:51:30 +08:00
Max Gekk cd649e7aef [SPARK-27793][SQL] Add ANSI SQL day-time and year-month interval types
### What changes were proposed in this pull request?
In the PR, I propose to extend Catalyst's type system by two new types that conform to the SQL standard (see SQL:2016, section 4.6.3):
- `DayTimeIntervalType` represents the day-time interval type,
- `YearMonthIntervalType` for SQL year-month interval type.

This PR only adds the two new DataType implementations, and there will be more PRs as sub-tasks of SPARK-27790 to completely support the new ANSI interval types.

### Why are the changes needed?
Spark as it is today supports an INTERVAL datatype. However this type is of very limited use. Existing interval values cannot be compared with any other interval values, or persisted to storage. Spark users request to either implement new or expand existing built-in functions which produce some sort of measures for elapsed time, such as `DATEDIFF()`. Rather than work around the edges to fill the potholes of the existing INTERVAL data type, I would like to propose to deliver a proper ANSI compliant INTERVAL type that can be introduced with minimal incompatibility, is comparable and thus sortable, and can be persisted in tables.

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

### How was this patch tested?
1. By checking coding style via:
```
$ ./dev/scalastyle
$ ./dev/lint-java
```
2. Run the test for the default sizes:
```
$ build/sbt "test:testOnly *DataTypeSuite"
```

Closes #31614 from MaxGekk/day-time-interval-type.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 04:44:23 +00:00
Cheng Su 5362f08125 [SPARK-34593][SQL] Preserve broadcast nested loop join partitioning and ordering
### What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` does not preserve `outputPartitioning` and `outputOrdering` right now. But it can preserve the streamed side partitioning and ordering when possible. This can help avoid shuffle and sort in later stage, if there's join and aggregation in the query. See example queries in added unit test in `JoinSuite.scala`.

In addition, fix a bunch of minor places in `BroadcastNestedLoopJoinExec.scala` for better style and readability.

### Why are the changes needed?

Avoid shuffle and sort for certain complicated query shape. Better query performance can be achieved.

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

No.

### How was this patch tested?

Added unit test in `JoinSuite.scala`.

Closes #31708 from c21/nested-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 04:32:28 +00:00
Kris Mok ecf4811764 [SPARK-34596][SQL] Use Utils.getSimpleName to avoid hitting Malformed class name in NewInstance.doGenCode
### What changes were proposed in this pull request?

Use `Utils.getSimpleName` to avoid hitting `Malformed class name` error in `NewInstance.doGenCode`.

### Why are the changes needed?

On older JDK versions (e.g. JDK8u), nested Scala classes may trigger `java.lang.Class.getSimpleName` to throw an `java.lang.InternalError: Malformed class name` error.
In this particular case, creating an `ExpressionEncoder` on such a nested Scala class would create a `NewInstance` expression under the hood, which will trigger the problem during codegen.

Similar to https://github.com/apache/spark/pull/29050, we should use  Spark's `Utils.getSimpleName` utility function in place of `Class.getSimpleName` to avoid hitting the issue.

There are two other occurrences of `java.lang.Class.getSimpleName` in the same file, but they're safe because they're only guaranteed to be only used on Java classes, which don't have this problem, e.g.:
```scala
    // Make a copy of the data if it's unsafe-backed
    def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) =
      s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value"
    val genFunctionValue: String = lambdaFunction.dataType match {
      case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value)
      case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value)
      case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value)
      case _ => genFunction.value
    }
```
The Unsafe-* family of types are all Java types, so they're okay.

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

Fixes a bug that throws an error when using `ExpressionEncoder` on some nested Scala types, otherwise no changes.

### How was this patch tested?

Added a test case to `org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite`. It'll fail on JDK8u before the fix, and pass after the fix.

Closes #31709 from rednaxelafx/spark-34596-master.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 12:22:51 +09:00
Liang-Chi Hsieh 107766661a [SPARK-34548][SQL][FOLLOW-UP] Call toSeq to recover Scala 2.13 build in RemoveNoopUnion
### What changes were proposed in this pull request?

Call `toSeq` to fix Scala 2.13 build error.

### Why are the changes needed?

It is needed to fix 2.13 build error.

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

No.

### How was this patch tested?

Existing tests.

Closes #31716 from viirya/SPARK-34548-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 11:12:58 +09:00
Liang-Chi Hsieh bab9531134 [SPARK-34548][SQL] Remove unnecessary children from Union under Distince and Deduplicate
### What changes were proposed in this pull request?

This patch proposes to remove unnecessary children from Union under Distince and Deduplicate

### Why are the changes needed?

If there are any duplicate child of `Union` under `Distinct` and `Deduplicate`, it can be removed to simplify query plan.

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

No

### How was this patch tested?

Unit test

Closes #31656 from viirya/SPARK-34548.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-02 17:09:08 -08:00
Kent Yao 6093a78dbd [SPARK-34558][SQL] warehouse path should be qualified ahead of populating and use
### What changes were proposed in this pull request?

Currently, the warehouse path gets fully qualified in the caller side for creating a database, table, partition, etc. An unqualified path is populated into Spark and Hadoop confs, which leads to inconsistent API behaviors.  We should make it qualified ahead.

When the value is a relative path `spark.sql.warehouse.dir=lakehouse`, some behaviors become inconsistent, for example.

If the default database is absent at runtime, the app fails with

```java
Caused by: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:./lakehouse
	at org.apache.hadoop.fs.Path.initialize(Path.java:263)
	at org.apache.hadoop.fs.Path.<init>(Path.java:254)
	at org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:133)
	at org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:137)
	at org.apache.hadoop.hive.metastore.Warehouse.getWhRoot(Warehouse.java:150)
	at org.apache.hadoop.hive.metastore.Warehouse.getDefaultDatabasePath(Warehouse.java:163)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB_core(HiveMetaStore.java:636)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:655)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:431)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invokeInternal(RetryingHMSHandler.java:148)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:107)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:79)
	... 73 more
```

If the default database is present at runtime, the app can work with it, and if we create a database, it gets fully qualified, for example

```sql
spark-sql> create database test;
Time taken: 0.052 seconds
spark-sql> desc database test;
Database Name	test
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210226/lakehouse/test.db
Owner	kentyao
Time taken: 0.023 seconds, Fetched 4 row(s)
```

Another thing is that the log becomes nubilous, for example.

```logtalk
21/02/27 13:54:17 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir ('datalake').
21/02/27 13:54:17 INFO SharedState: Warehouse path is 'lakehouse'.
```

### Why are the changes needed?

fix bug and ambiguity
### Does this PR introduce _any_ user-facing change?

yes, the path now resolved with proper order - `warehouse->database->table->partition`

### How was this patch tested?

w/ ut added

Closes #31671 from yaooqinn/SPARK-34558.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 15:14:19 +00:00
kevincmchen 9e8547ca43 [SPARK-34498][SQL][TESTS] fix the remaining problems in #31560
### What changes were proposed in this pull request?

This is a followup of #31560,
In  #31560,  we added `JavaSimpleWritableDataSource ` and left some little problems like unused interface `SessionConfigSupport` 、 inconsistent schema between `JavaSimpleWritableDataSource ` and `SimpleWritableDataSource`.
This PR fixes the remaining problems in #31560.

### Why are the changes needed?

1. `SessionConfigSupport` in `JavaSimpleWritableDataSource ` and `SimpleWritableDataSource` is never used, so we don't need to implement it.
2. change the schema of `SimpleWritableDataSource`, to match `TestingV2Source`

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

NO

### How was this patch tested?

existing testsuites

Closes #31621 from kevincmchen/SPARK-34498.

Authored-by: kevincmchen <kevincmchen@tencent.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 15:08:23 +00:00
Karen Feng 2e54d68eb9 [SPARK-34547][SQL] Only use metadata columns for resolution as last resort
### What changes were proposed in this pull request?

Today, child expressions may be resolved based on "real" or metadata output attributes. We should prefer the real attribute during resolution if one exists.

### Why are the changes needed?

Today, attempting to resolve an expression when there is a "real" output attribute and a metadata attribute with the same name results in resolution failure. This is likely unexpected, as the user may not know about the metadata attribute.

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

Yes. Previously, the user would see an error message when resolving a column with the same name as a "real" output attribute and a metadata attribute as below:
```
org.apache.spark.sql.AnalysisException: Reference 'index' is ambiguous, could be: testcat.ns1.ns2.tableTwo.index, testcat.ns1.ns2.tableOne.index.; line 1 pos 71
at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:363)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:107)
```

Now, resolution succeeds and provides the "real" output attribute.

### How was this patch tested?

Added a unit test.

Closes #31654 from karenfeng/fallback-resolve-metadata.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 17:27:13 +08:00
Amandeep Sharma 4bda3c0f02 [SPARK-34417][SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot
**What changes were proposed in this pull request?**

This PR fixes dataframe.na.fillMap() for column having a dot in the name as mentioned in [SPARK-34417](https://issues.apache.org/jira/browse/SPARK-34417).

Use resolved attributes of a column for replacing null values.

**Why are the changes needed?**
dataframe.na.fillMap() does not work for column having a dot in the name

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

**How was this patch tested?**
Added unit test for the same

Closes #31545 from amandeep-sharma/master.

Lead-authored-by: Amandeep Sharma <happyaman91@gmail.com>
Co-authored-by: Amandeep Sharma <amandeep.sharma@oracle.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 17:14:15 +08:00
Anton Okolnychyi 08a125761d [SPARK-34585][SQL] Remove no longer needed BatchWriteHelper
### What changes were proposed in this pull request?

As a follow-up to SPARK-34456, this PR removes `BatchWriteHelper` completely.

### Why are the changes needed?

These changes remove no longer used code.

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

No.

### How was this patch tested?

Existing tests.

Closes #31699 from aokolnychyi/spark-34585.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:58:18 +09:00
Dongjoon Hyun 4818847e87 [SPARK-34578][SQL][TESTS][TEST-MAVEN] Refactor ORC encryption tests and ignore ORC shim loaded by old Hadoop library
### What changes were proposed in this pull request?

1. This PR aims to ignore ORC encryption tests when ORC shim is loaded by old Hadoop library by some other tests. The test coverage is preserved by Jenkins SBT runs and GitHub Action jobs. This PR only aims to recover Maven Jenkins jobs.
2. In addition, this PR simplifies SBT testing by refactor the test config to `SparkBuild.scala/pom.xml` and remove `DedicatedJVMTest`. This will remove one GitHub Action job which was recently added for `DedicatedJVMTest` tag.

### Why are the changes needed?

Currently, Maven test fails when it runs in a batch mode because `HadoopShimsPre2_3$NullKeyProvider` is loaded.

**MVN COMMAND**
```
$ mvn test -pl sql/core --am -Dtest=none -DwildcardSuites=org.apache.spark.sql.execution.datasources.orc.OrcV1QuerySuite,org.apache.spark.sql.execution.datasources.orc.OrcEncryptionSuite
```

**BEFORE**
```
- Write and read an encrypted table *** FAILED ***
...
  Cause: 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 1) (localhost executor driver): java.lang.IllegalArgumentException: Unknown key pii
	at org.apache.orc.impl.HadoopShimsPre2_3$NullKeyProvider.getCurrentKeyVersion(HadoopShimsPre2_3.java:71)
	at org.apache.orc.impl.WriterImpl.getKey(WriterImpl.java:871)
```

**AFTER**
```
OrcV1QuerySuite
...
OrcEncryptionSuite:
- Write and read an encrypted file !!! CANCELED !!!
  [] was empty org.apache.orc.impl.HadoopShimsPre2_3$NullKeyProvider1b705f65 doesn't has the test keys. ORC shim is created with old Hadoop libraries (OrcEncryptionSuite.scala:39)
- Write and read an encrypted table !!! CANCELED !!!
  [] was empty org.apache.orc.impl.HadoopShimsPre2_3$NullKeyProvider22adeee1 doesn't has the test keys. ORC shim is created with old Hadoop libraries (OrcEncryptionSuite.scala:67)
```

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

No.

### How was this patch tested?

Pass the Jenkins Maven tests.

For SBT command,
- the test suite required a dedicated JVM (Before)
- the test suite doesn't require a dedicated JVM (After)
```
$ build/sbt "sql/testOnly *.OrcV1QuerySuite *.OrcEncryptionSuite"
...
[info] OrcV1QuerySuite
...
[info] - SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core (26 milliseconds)
[info] OrcEncryptionSuite:
[info] - Write and read an encrypted file (431 milliseconds)
[info] - Write and read an encrypted table (359 milliseconds)
[info] All tests passed.
[info] Passed: Total 35, Failed 0, Errors 0, Passed 35
```

Closes #31697 from dongjoon-hyun/SPARK-34578-TEST.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:52:27 +09:00
Chao Sun ce13dcc689 [SPARK-32703][SQL] Replace deprecated API calls from SpecificParquetRecordReaderBase
### What changes were proposed in this pull request?

Currently in `SpecificParquetRecordReaderBase` we use deprecated APIs in a few places from Parquet, such as `readFooter`, `ParquetInputSplit`, `new ParquetFileReader`, `filterRowGroups`, etc. This replaces these with the newer APIs. In specific this:
- Replaces `ParquetInputSplit` with `FileSplit`. We never use specific things in the former such as `rowGroupOffsets` so the swap is pretty simple.
- Removes `readFooter` calls by using `ParquetFileReader.open`
- Replace deprecated `ParquetFileReader` ctor with the newer API which takes `ParquetReadOptions`.
- Removes the unnecessary handling of case when `rowGroupOffsets` is not null. It seems this never happens.

### Why are the changes needed?

The aforementioned APIs were deprecated and is going to be removed at some point in future. This is to ensure better supportability.

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

No

### How was this patch tested?

This is a cleanup and relies on existing tests on the relevant code paths.

Closes #31667 from sunchao/SPARK-32703.

Lead-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:51:41 +09:00
Richard Penney 7d0743b493 [SPARK-33678][SQL] Product aggregation function
### Why is this change being proposed?
This patch adds support for a new "product" aggregation function in `sql.functions` which multiplies-together all values in an aggregation group.

This is likely to be useful in statistical applications which involve combining probabilities, or financial applications that involve combining cumulative interest rates, but is also a versatile mathematical operation of similar status to `sum` or `stddev`. Other users [have noted](https://stackoverflow.com/questions/52991640/cumulative-product-in-spark) the absence of such a function in current releases of Spark.

This function is both much more concise than an expression of the form `exp(sum(log(...)))`, and avoids awkward edge-cases associated with some values being zero or negative, as well as being less computationally costly.

### Does this PR introduce _any_ user-facing change?
No - only adds new function.

### How was this patch tested?
Built-in tests have been added for the new `catalyst.expressions.aggregate.Product` class and its invocation via the (scala) `sql.functions.product` function. The latter, and the PySpark wrapper have also been manually tested in spark-shell and pyspark sessions. The SparkR wrapper is currently untested, and may need separate validation (I'm not an "R" user myself).

An illustration of the new functionality, within PySpark is as follows:
```
import pyspark.sql.functions as pf, pyspark.sql.window as pw

df = sqlContext.range(1, 17).toDF("x")
win = pw.Window.partitionBy(pf.lit(1)).orderBy(pf.col("x"))

df.withColumn("factorial", pf.product("x").over(win)).show(20, False)
+---+---------------+
|x  |factorial      |
+---+---------------+
|1  |1.0            |
|2  |2.0            |
|3  |6.0            |
|4  |24.0           |
|5  |120.0          |
|6  |720.0          |
|7  |5040.0         |
|8  |40320.0        |
|9  |362880.0       |
|10 |3628800.0      |
|11 |3.99168E7      |
|12 |4.790016E8     |
|13 |6.2270208E9    |
|14 |8.71782912E10  |
|15 |1.307674368E12 |
|16 |2.0922789888E13|
+---+---------------+
```

Closes #30745 from rwpenney/feature/agg-product.

Lead-authored-by: Richard Penney <rwp@rwpenney.uk>
Co-authored-by: Richard Penney <rwpenney@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:51:07 +09:00
Gabriele Nizzoli b13a4b85d4 [SPARK-34573][SQL] Avoid global locking in SQLConf object for sqlConfEntries map
### What changes were proposed in this pull request?
In the `SQLConf` object, the `sqlConfEntries` map is globally synchronized (it is a Java `Collections.synchronizedMap`): any operation, including a get, will need to acquire the lock.

An example of this is calling the `DatatType.sameType` method. This will trigger a check on `SQLConf.get.caseSensitiveAnalysis`. So every time we compare two datatypes with sameType, we hit a lock.

To avoid having multiple tasks locking on this, a better approach would be to use a map that does not lock on read (like a `ConcurrentHashMap`). This map implementation does not lock on read, and on write it only locks the map partially. The only lock that happens is on write on the same map key.

### Why are the changes needed?
Multiple tasks performing any operation that directly or indirectly trigger a query to the `SQLConf.sqlConfEntries` map, will require acquiring a global lock on that map. Something as easy as calling `DataType.sameType(...)` would be locking on the global `sqlConfEntries` lock of the `Collections.synchronizedMap`.

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

### How was this patch tested?
No functionality change. Existing unit tests run normally.

Closes #31689 from gabrielenizzoli/SPARK-34573.

Authored-by: Gabriele Nizzoli <1545350+gabrielenizzoli@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-02 15:36:51 +09:00
Max Gekk 70f6267de6 [SPARK-34560][SQL] Generate unique output attributes in the SHOW TABLES logical node
### What changes were proposed in this pull request?
In the PR, I propose to generate unique attributes in the logical nodes of the `SHOW TABLES` command.

Also, this PR fixes similar issues in other logical nodes:
- ShowTableExtended
- ShowViews
- ShowTableProperties
- ShowFunctions
- ShowColumns
- ShowPartitions
- ShowNamespaces

### Why are the changes needed?
This fixes the issue which is demonstrated by the example below:
```scala
scala> val show1 = sql("SHOW TABLES IN ns1")
show1: org.apache.spark.sql.DataFrame = [namespace: string, tableName: string ... 1 more field]

scala> val show2 = sql("SHOW TABLES IN ns2")
show2: org.apache.spark.sql.DataFrame = [namespace: string, tableName: string ... 1 more field]

scala> show1.show
+---------+---------+-----------+
|namespace|tableName|isTemporary|
+---------+---------+-----------+
|      ns1|     tbl1|      false|
+---------+---------+-----------+

scala> show2.show
+---------+---------+-----------+
|namespace|tableName|isTemporary|
+---------+---------+-----------+
|      ns2|     tbl2|      false|
+---------+---------+-----------+

scala> show1.join(show2).where(show1("tableName") =!= show2("tableName")).show
org.apache.spark.sql.AnalysisException: Column tableName#17 are ambiguous. It's probably because you joined several Datasets together, and some of these Datasets are the same. This column points to one of the Datasets but Spark is unable to figure out which one. Please alias the Datasets with different names via `Dataset.as` before joining them, and specify the column using qualified name, e.g. `df.as("a").join(df.as("b"), $"a.id" > $"b.id")`. You can also set spark.sql.analyzer.failAmbiguousSelfJoin to false to disable this check.
  at org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin$.apply(DetectAmbiguousSelfJoin.scala:157)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works as expected:
```scala
scala> show1.join(show2).where(show1("tableName") =!= show2("tableName")).show
+---------+---------+-----------+---------+---------+-----------+
|namespace|tableName|isTemporary|namespace|tableName|isTemporary|
+---------+---------+-----------+---------+---------+-----------+
|      ns1|     tbl1|      false|      ns2|     tbl2|      false|
+---------+---------+-----------+---------+---------+-----------+
```

### How was this patch tested?
By running the new test:
```
$  build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
```

Closes #31675 from MaxGekk/fix-output-attrs.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-01 18:32:32 +00:00
Max Gekk 984ff396a2 [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 DESCRIBE TABLE
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the `DESCRIBE TABLE` command.

### Why are the changes needed?
This fixes the issue demonstrated by the example:
```scala
val tbl = "testcat.ns1.ns2.tbl"
sql(s"CREATE TABLE $tbl (c0 INT) USING _")
val description = sql(s"DESCRIBE TABLE $tbl")
description.drop("comment")
```
The `drop()` method fails with the error:
```
org.apache.spark.sql.AnalysisException: Resolved attribute(s) col_name#102,data_type#103 missing from col_name#29,data_type#30,comment#31 in operator !Project [col_name#102, data_type#103]. Attribute(s) with the same name appear in the operation: col_name,data_type. Please check if the right attribute(s) are used.;
!Project [col_name#102, data_type#103]
+- LocalRelation [col_name#29, data_type#30, comment#31]

	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:51)
	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:50)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, `drop()`/`add()` works as expected:
```scala
description.drop("comment").show()
+---------------+---------+
|       col_name|data_type|
+---------------+---------+
|             c0|      int|
|               |         |
| # Partitioning|         |
|Not partitioned|         |
+---------------+---------+
```

### How was this patch tested?
1. Run new test:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
```
2. Run existing test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31676 from MaxGekk/describe-table-drop-column.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-01 22:20:28 +08:00
Shixiong Zhu 62737e140c [SPARK-34556][SQL] Checking duplicate static partition columns should respect case sensitive conf
### What changes were proposed in this pull request?

This PR makes partition spec parsing respect case sensitive conf.

### Why are the changes needed?

When parsing the partition spec, Spark will call `org.apache.spark.sql.catalyst.parser.ParserUtils.checkDuplicateKeys` to check if there are duplicate partition column names in the list. But this method is always case sensitive and doesn't detect duplicate partition column names when using different cases.

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

Yep. This prevents users from writing incorrect queries such as `INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)` when they don't enable case sensitive conf.

### How was this patch tested?

The new added test will fail without this change.

Closes #31669 from zsxwing/SPARK-34556.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-01 13:55:35 +09:00
Kent Yao 1afe284ed8 [SPARK-34570][SQL] Remove dead code from constructors of [Hive]SessionStateBuilder
### What changes were proposed in this pull request?

the parameter - `options` is never used. The changes here was part of https://github.com/apache/spark/pull/30642, It got reverted for easier backporting #30642 as a hotfix by dad24543aa, this PR brings it back to master.

### Why are the changes needed?

remove unless dead code

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

no

### How was this patch tested?

Passing CI is enough.

Closes #31683 from yaooqinn/SPARK-34570.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:30:18 +09:00
Angerszhuuuu d574308864 [SPARK-34579][SQL][TEST] Fix wrong UT in SQLQuerySuite
### What changes were proposed in this pull request?
Some UT in SQLQuerySuite is  not incorrect, it have wrong table name in `withTable`, this pr to make it correct.

### Why are the changes needed?
Fix UT

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

### How was this patch tested?
Existed UT

Closes #31681 from AngersZhuuuu/SPARK-34569.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-02-28 16:21:42 -08:00
Shardul Mahadik 0216051aca [SPARK-34506][CORE] ADD JAR with ivy coordinates should be compatible with Hive transitive behavior
### What changes were proposed in this pull request?
SPARK-33084 added the ability to use ivy coordinates with `SparkContext.addJar`. PR #29966 claims to mimic Hive behavior although I found a few cases where it doesn't

1) The default value of the transitive parameter is false, both in case of parameter not being specified in coordinate or parameter value being invalid. The Hive behavior is that transitive is [true if not specified](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L169)) in the coordinate and [false for invalid values](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L124)). Also, regardless of Hive, I think a default of true for the transitive parameter also matches [ivy's own defaults](https://ant.apache.org/ivy/history/2.5.0/ivyfile/dependency.html#_attributes).

2) The parameter value for transitive parameter is regarded as case-sensitive [based on the understanding](https://github.com/apache/spark/pull/29966#discussion_r547752259) that Hive behavior is case-sensitive. However, this is not correct, Hive [treats the parameter value case-insensitively](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L122)).

I propose that we be compatible with Hive for these behaviors

### Why are the changes needed?
To make `ADD JAR` with ivy coordinates compatible with Hive's transitive behavior

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

The user-facing changes here are within master as the feature introduced in SPARK-33084 has not been released yet
1. Previously an ivy coordinate without `transitive` parameter specified did not resolve transitive dependency, now it does.
2. Previously an `transitive` parameter value was treated case-sensitively. e.g. `transitive=TRUE` would be treated as false as it did not match exactly `true`. Now it will be treated case-insensitively.

### How was this patch tested?

Modified existing unit tests to test new behavior
Add new unit test to cover usage of `exclude` with unspecified `transitive`

Closes #31623 from shardulm94/spark-34506.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:10:20 +09:00
Yuming Wang d07fc3076b [SPARK-33687][SQL] Support analyze all tables in a specific database
### What changes were proposed in this pull request?

This pr add support analyze all tables in a specific database:
```g4
 ANALYZE TABLES ((FROM | IN) multipartIdentifier)? COMPUTE STATISTICS (identifier)?
```

### Why are the changes needed?

1. Make it easy to analyze all tables in a specific database.
2. PostgreSQL has a similar implementation: https://www.postgresql.org/docs/12/sql-analyze.html.

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

No.

### How was this patch tested?

The feature tested by unit test.
The documentation tested by regenerating the documentation:

menu-sql.yaml |  sql-ref-syntax-aux-analyze-tables.md
-- | --
![image](https://user-images.githubusercontent.com/5399861/109098769-dc33a200-775c-11eb-86b1-55531e5425e0.png) | ![image](https://user-images.githubusercontent.com/5399861/109098841-02594200-775d-11eb-8588-de8da97ec94a.png)

Closes #30648 from wangyum/SPARK-33687.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:06:47 +09:00
Yuming Wang 54c053afb0 [SPARK-34479][SQL] Add zstandard codec to Avro compression codec list
### What changes were proposed in this pull request?

Avro add zstandard codec since AVRO-2195. This pr add zstandard codec to Avro compression codec list.

### Why are the changes needed?

To make Avro support zstandard codec.

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

No.

### How was this patch tested?

Unit test.

Closes #31673 from wangyum/SPARK-34479.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-02-27 10:31:42 -08:00
Ruifeng Zheng 05069ff4ce [SPARK-34353][SQL] CollectLimitExec avoid shuffle if input rdd has 0/1 partition
### What changes were proposed in this pull request?
if child rdd has only one partition or zero partition, skip the shuffle

### Why are the changes needed?
skip shuffle if possible

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

### How was this patch tested?
existing testsuites

Closes #31468 from zhengruifeng/collect_limit_single_partition.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-27 16:48:20 +09:00
ShiKai Wang 56e664c717 [SPARK-34392][SQL] Support ZoneOffset +h:mm in DateTimeUtils. getZoneId
### What changes were proposed in this pull request?
To support +8:00 in Spark3 when execute sql
`select to_utc_timestamp("2020-02-07 16:00:00", "GMT+8:00")`

### Why are the changes needed?
+8:00 this format is supported in PostgreSQL,hive, presto, but not supported in Spark3
https://issues.apache.org/jira/browse/SPARK-34392

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

### How was this patch tested?
unit test

Closes #31624 from Karl-WangSK/zone.

Lead-authored-by: ShiKai Wang <wskqing@gmail.com>
Co-authored-by: Karl-WangSK <shikai.wang@linkflowtech.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-26 11:03:20 -06:00
tanel.kiis@gmail.com 67ec4f7f67 [SPARK-33971][SQL] Eliminate distinct from more aggregates
### What changes were proposed in this pull request?

Add more aggregate expressions to `EliminateDistinct` rule.

### Why are the changes needed?

Distinct aggregation can add a significant overhead. It's better to remove distinct whenever possible.

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

No

### How was this patch tested?

UT

Closes #30999 from tanelk/SPARK-33971_eliminate_distinct.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-26 21:59:02 +09:00
Max Gekk c1beb16cc8 [SPARK-34554][SQL] Implement the copy() method in ColumnarMap
### What changes were proposed in this pull request?
Implement `ColumnarMap.copy()` by using the `copy()` method of `ColumnarArray`.

### Why are the changes needed?
To eliminate `java.lang.UnsupportedOperationException` while using `ColumnarMap`.

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

### How was this patch tested?
By running new tests in `ColumnarBatchSuite`.

Closes #31663 from MaxGekk/columnar-map-copy.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-26 21:33:14 +09:00
ulysses-you 82267acfe8 [SPARK-34550][SQL] Skip InSet null value during push filter to Hive metastore
### What changes were proposed in this pull request?

Skip `InSet` null value during push filter to Hive metastore.

### Why are the changes needed?

If `InSet` contains a null value, we should skip it and push other values to metastore. To keep same behavior with `In`.

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

No.

### How was this patch tested?

Add test.

Closes #31659 from ulysses-you/SPARK-34550.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-26 21:29:14 +09:00
Cheng Su 7d5021f5ee [SPARK-34533][SQL] Eliminate LEFT ANTI join to empty relation in AQE
### What changes were proposed in this pull request?

I discovered from review discussion - https://github.com/apache/spark/pull/31630#discussion_r581774000 , that we can eliminate LEFT ANTI join (with no join condition) to empty relation, if the right side is known to be non-empty. So with AQE, this is doable similar to https://github.com/apache/spark/pull/29484 .

### Why are the changes needed?

This can help eliminate the join operator during logical plan optimization.
Before this PR, [left side physical plan `execute()` will be called](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala#L192), so if left side is complicated (e.g. contain broadcast exchange operator), then some computation would happen. However after this PR, the join operator will be removed during logical plan, and nothing will be computed from left side. Potentially it can save resource for these kinds of query.

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

No.

### How was this patch tested?

Added unit tests for positive and negative queries in `AdaptiveQueryExecSuite.scala`.

Closes #31641 from c21/left-anti-aqe.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-26 11:46:27 +00:00
Wenchen Fan 73857cdd87 [SPARK-34524][SQL] Simplify v2 partition commands resolution
### What changes were proposed in this pull request?

This PR simplifies the resolution of v2 partition commands:
1. Add a common trait for v2 partition commands, so that we don't need to match them one by one in the rules.
2. Make partition spec an expression, so that it's easier to resolve them via tree node transformation.
3. Add `TruncatePartition` so that `TruncateTable` doesn't need to be a v2 partition command.
4. Simplify `CheckAnalysis` to only check if the table is partitioned. For partitioned tables, partition spec is always resolved, so we don't need to check it. The `SupportsAtomicPartitionManagement` check is also done in the runtime. Since Spark eagerly executes commands, exception in runtime will also be thrown at analysis time.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31637 from cloud-fan/simplify.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-26 11:44:42 +00:00
Max Gekk 5c7d019b60 [SPARK-34543][SQL] Respect the spark.sql.caseSensitive config while resolving partition spec in v1 SET LOCATION
### What changes were proposed in this pull request?
Preprocess the partition spec passed to the V1 `ALTER TABLE .. SET LOCATION` implementation `AlterTableSetLocationCommand`, and normalize the passed spec according to the partition columns w.r.t the case sensitivity flag  **spark.sql.caseSensitive**.

### Why are the changes needed?
V1 `ALTER TABLE .. SET LOCATION` is case sensitive in fact, and doesn't respect the SQL config **spark.sql.caseSensitive** which is false by default, for instance:
```sql
spark-sql> CREATE TABLE tbl (id INT, part INT) PARTITIONED BY (part);
spark-sql> INSERT INTO tbl PARTITION (part=0) SELECT 0;
spark-sql> SHOW TABLE EXTENDED LIKE 'tbl' PARTITION (part=0);
Location: file:/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/part=0
spark-sql> ALTER TABLE tbl ADD PARTITION (part=1);
spark-sql> SELECT * FROM tbl;
0	0
```
Create new partition folder in the file system:
```
$ cp -r /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/part=0 /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/aaa
```
Set new location for the partition part=1:
```sql
spark-sql> ALTER TABLE tbl PARTITION (part=1) SET LOCATION '/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/aaa';
spark-sql> SELECT * FROM tbl;
0	0
0	1
spark-sql> ALTER TABLE tbl ADD PARTITION (PART=2);
spark-sql> SELECT * FROM tbl;
0	0
0	1
```
Set location for a partition in the upper case:
```
$ cp -r /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/part=0 /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/bbb
```
```sql
spark-sql> ALTER TABLE tbl PARTITION (PART=2) SET LOCATION '/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/bbb';
Error in query: Partition spec is invalid. The spec (PART) must match the partition spec (part) defined in table '`default`.`tbl`'
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the command above works as expected:
```sql
spark-sql> ALTER TABLE tbl PARTITION (PART=2) SET LOCATION '/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/bbb';
spark-sql> SELECT * FROM tbl;
0	0
0	1
0	2
```

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31651 from MaxGekk/set-location-case-sense.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-26 15:28:57 +08:00
yangjie01 0d3a9cd3c9 [SPARK-34535][SQL] Cleanup unused symbol in Orc related code
### What changes were proposed in this pull request?
Cleanup unused symbol in Orc related code as follows:

- `OrcDeserializer` : parameter `dataSchema` in constructor
- `OrcFilters`  : parameter `schema ` in method `convertibleFilters`.
- `OrcPartitionReaderFactory`: ignore return value of `OrcUtils.orcResultSchemaString` in  method `buildReader(file: PartitionedFile)`

### Why are the changes needed?
Cleanup code.

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

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

Closes #31644 from LuciferYang/cleanup-orc-unused-symbol.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-26 09:20:40 +09:00
Wenchen Fan dffb01f28a [SPARK-34152][SQL][FOLLOWUP] Do not uncache the temp view if it doesn't exist
### What changes were proposed in this pull request?

This PR fixes a mistake in https://github.com/apache/spark/pull/31273. When CREATE OR REPLACE a temp view, we need to uncache the to-be-replaced existing temp view. However, we shouldn't uncache if there is no existing temp view.

This doesn't cause real issues because the uncache action is failure-safe. But it produces a lot of warning messages.

### Why are the changes needed?

Avoid unnecessary warning logs.

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

no

### How was this patch tested?

manually run tests and check the warning messages.

Closes #31650 from cloud-fan/warnning.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-25 15:25:41 -08:00
Liang-Chi Hsieh f7ac2d655c [SPARK-34474][SQL] Remove unnecessary Union under Distinct/Deduplicate
### What changes were proposed in this pull request?

This patch proposes to let optimizer to remove unnecessary `Union` under `Distinct`/`Deduplicate`.

### Why are the changes needed?

For an `Union` under `Distinct`/`Deduplicate`, if its children are all the same, we can just keep one among them and remove the `Union`.

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

No

### How was this patch tested?

Unit tests.

Closes #31595 from viirya/remove-union.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-25 12:41:07 -08:00
Yuming Wang 4a3200b08a [SPARK-34436][SQL] DPP support LIKE ANY/ALL expression
### What changes were proposed in this pull request?

This pr make DPP support LIKE ANY/ALL expression:
```sql
SELECT date_id, product_id FROM fact_sk f
JOIN dim_store s
ON f.store_id = s.store_id WHERE s.country LIKE ANY ('%D%E%', '%A%B%')
```

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31563 from wangyum/SPARK-34436.

Lead-authored-by: Yuming Wang <yumwang@apache.org>
Co-authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-25 18:07:39 +08:00
Max Gekk c56af69cdf [SPARK-34518][SQL] Rename AlterTableRecoverPartitionsCommand to RepairTableCommand
### What changes were proposed in this pull request?
Rename the execution node `AlterTableRecoverPartitionsCommand` for the commands:
- `MSCK REPAIR TABLE table [{ADD|DROP|SYNC} PARTITIONS]`
- `ALTER TABLE table RECOVER PARTITIONS`

to `RepairTableCommand`.

### Why are the changes needed?
1. After the PR https://github.com/apache/spark/pull/31499, `ALTER TABLE table RECOVER PARTITIONS` is equal to `MSCK REPAIR TABLE table ADD PARTITIONS`. And mapping of the generic command `MSCK REPAIR TABLE` to the more specific execution node `AlterTableRecoverPartitionsCommand` can confuse devs in the future.
2. `ALTER TABLE table RECOVER PARTITIONS` does not support any options/extensions. So, additional parameters `enableAddPartitions` and `enableDropPartitions` in `AlterTableRecoverPartitionsCommand` confuse as well.

### Does this PR introduce _any_ user-facing change?
No because this is internal API.

### How was this patch tested?
By running the existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsSuite"
$ build/sbt "test:testOnly *AlterTableRecoverPartitionsParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *MsckRepairTableSuite"
$ build/sbt "test:testOnly *MsckRepairTableParserSuite"
```

Closes #31635 from MaxGekk/rename-recover-partitions.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-25 09:32:41 +00:00
Gabor Somogyi 44eadb943b [SPARK-34497][SQL] Fix built-in JDBC connection providers to restore JVM security context changes
### What changes were proposed in this pull request?
Some of the built-in JDBC connection providers are changing the JVM security context to do the authentication which is fine. The problematic part is that executors can be reused by another query. The following situation leads to incorrect behaviour:
* Query1 opens JDBC connection and changes JVM security context in Executor1
* Query2 tries to open JDBC connection but it realizes there is already an entry for that DB type in Executor1
* Query2 is not changing JVM security context and uses Query1 keytab and principal
* Query2 fails with authentication error

In this PR I've changed to code such a way that JVM security context is changed all the time but only temporarily until the connection built-up and then rolled back. Since `getConnection` is synchronised with `SecurityConfigurationLock` it ends-up in correct behaviour without any race.

### Why are the changes needed?
Incorrect JVM security context handling.

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

### How was this patch tested?
Existing unit + integration tests.

Closes #31622 from gaborgsomogyi/SPARK-34497.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-25 09:25:17 +09:00