Commit graph

4712 commits

Author SHA1 Message Date
Peter Toth 3309a2be07 [SPARK-32635][SQL][FOLLOW-UP] Add a new test case in catalyst module
### What changes were proposed in this pull request?
This is a follow-up PR to https://github.com/apache/spark/pull/29771 and just adds a new test case.

### Why are the changes needed?
To have better test coverage.

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

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

Closes #29802 from peter-toth/SPARK-32635-fix-foldable-propagation-followup.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-18 13:56:19 -07:00
yangjie01 2128c4f14b [SPARK-32808][SQL] Pass all test of sql/core module in Scala 2.13
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/29660 and https://github.com/apache/spark/pull/29689 there are 13 remaining  failed cases of sql core module with Scala 2.13.

The reason for the remaining failed cases is the optimization result of `CostBasedJoinReorder` maybe different with same input in Scala 2.12 and Scala 2.13 if there are more than one same cost candidate plans.

In this pr give a way to make the  optimization result deterministic as much as possible to pass all remaining failed cases of `sql/core` module in Scala 2.13, the main change of this pr as follow:

- Change to use `LinkedHashMap` instead of `Map` to store `foundPlans` in `JoinReorderDP.search` method to ensure same iteration order with same insert order because iteration order of `Map` behave differently under Scala 2.12 and 2.13

- Fixed `StarJoinCostBasedReorderSuite` affected by the above change

- Regenerate golden files affected by the above change.

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: All tests passed.

Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/core -Pscala-2.13 -am
mvn test -pl sql/core -Pscala-2.13
```

**Before**
```
Tests: succeeded 8485, failed 13, canceled 1, ignored 52, pending 0
*** 13 TESTS FAILED ***

```

**After**

```
Tests: succeeded 8498, failed 0, canceled 1, ignored 52, pending 0
All tests passed.
```

Closes #29711 from LuciferYang/SPARK-32808-3.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-18 10:38:30 -05:00
gengjiaan 8b09536cdf [SPARK-27951][SQL] Support ANSI SQL NTH_VALUE window function
### What changes were proposed in this pull request?
The `NTH_VALUE` function is an ANSI SQL.
For examples:
```
CREATE TEMPORARY TABLE empsalary (
    depname varchar,
    empno bigint,
    salary int,
    enroll_date date
);

INSERT INTO empsalary VALUES
('develop', 10, 5200, '2007-08-01'),
('sales', 1, 5000, '2006-10-01'),
('personnel', 5, 3500, '2007-12-10'),
('sales', 4, 4800, '2007-08-08'),
('personnel', 2, 3900, '2006-12-23'),
('develop', 7, 4200, '2008-01-01'),
('develop', 9, 4500, '2008-01-01'),
('sales', 3, 4800, '2007-08-01'),
('develop', 8, 6000, '2006-10-01'),
('develop', 11, 5200, '2007-08-15');

select first_value(salary) over(order by salary range between 1000 preceding and 1000 following),
	lead(salary) over(order by salary range between 1000 preceding and 1000 following),
	nth_value(salary, 1) over(order by salary range between 1000 preceding and 1000 following),
	salary from empsalary;
 first_value | lead | nth_value | salary
-------------+------+-----------+--------
        3500 | 3900 |      3500 |   3500
        3500 | 4200 |      3500 |   3900
        3500 | 4500 |      3500 |   4200
        3500 | 4800 |      3500 |   4500
        3900 | 4800 |      3900 |   4800
        3900 | 5000 |      3900 |   4800
        4200 | 5200 |      4200 |   5000
        4200 | 5200 |      4200 |   5200
        4200 | 6000 |      4200 |   5200
        5000 |      |      5000 |   6000
(10 rows)
```

There are some mainstream database support the syntax.

**PostgreSQL:**
https://www.postgresql.org/docs/8.4/functions-window.html

**Vertica:**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/Analytic/NTH_VALUEAnalytic.htm?tocpath=SQL%20Reference%20Manual%7CSQL%20Functions%7CAnalytic%20Functions%7C_____23

**Oracle:**
https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/NTH_VALUE.html#GUID-F8A0E88C-67E5-4AA6-9515-95D03A7F9EA0

**Redshift**
https://docs.aws.amazon.com/redshift/latest/dg/r_WF_NTH.html

**Presto**
https://prestodb.io/docs/current/functions/window.html

**MySQL**
https://www.mysqltutorial.org/mysql-window-functions/mysql-nth_value-function/

### Why are the changes needed?
The `NTH_VALUE` function is an ANSI SQL.
The `NTH_VALUE` function is very useful.

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

### How was this patch tested?
Exists and new UT.

Closes #29604 from beliefer/support-nth_value.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-18 07:06:38 +00:00
Takeshi Yamamuro b49aaa33e1 [SPARK-32906][SQL] Struct field names should not change after normalizing floats
### What changes were proposed in this pull request?

This PR intends to fix a minor bug when normalizing floats for struct types;
```
scala> import org.apache.spark.sql.execution.aggregate.HashAggregateExec
scala> val df = Seq(Tuple1(Tuple1(-0.0d)), Tuple1(Tuple1(0.0d))).toDF("k")
scala> val agg = df.distinct()
scala> agg.explain()
== Physical Plan ==
*(2) HashAggregate(keys=[k#40], functions=[])
+- Exchange hashpartitioning(k#40, 200), true, [id=#62]
   +- *(1) HashAggregate(keys=[knownfloatingpointnormalized(if (isnull(k#40)) null else named_struct(col1, knownfloatingpointnormalized(normalizenanandzero(k#40._1)))) AS k#40], functions=[])
      +- *(1) LocalTableScan [k#40]

scala> val aggOutput = agg.queryExecution.sparkPlan.collect { case a: HashAggregateExec => a.output.head }
scala> aggOutput.foreach { attr => println(attr.prettyJson) }
### Final Aggregate ###
[ {
  "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
  "num-children" : 0,
  "name" : "k",
  "dataType" : {
    "type" : "struct",
    "fields" : [ {
      "name" : "_1",
                ^^^
      "type" : "double",
      "nullable" : false,
      "metadata" : { }
    } ]
  },
  "nullable" : true,
  "metadata" : { },
  "exprId" : {
    "product-class" : "org.apache.spark.sql.catalyst.expressions.ExprId",
    "id" : 40,
    "jvmId" : "a824e83f-933e-4b85-a1ff-577b5a0e2366"
  },
  "qualifier" : [ ]
} ]

### Partial Aggregate ###
[ {
  "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
  "num-children" : 0,
  "name" : "k",
  "dataType" : {
    "type" : "struct",
    "fields" : [ {
      "name" : "col1",
                ^^^^
      "type" : "double",
      "nullable" : true,
      "metadata" : { }
    } ]
  },
  "nullable" : true,
  "metadata" : { },
  "exprId" : {
    "product-class" : "org.apache.spark.sql.catalyst.expressions.ExprId",
    "id" : 40,
    "jvmId" : "a824e83f-933e-4b85-a1ff-577b5a0e2366"
  },
  "qualifier" : [ ]
} ]
```

### Why are the changes needed?

bugfix.

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

No.

### How was this patch tested?

Added tests.

Closes #29780 from maropu/FixBugInNormalizedFloatingNumbers.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-09-17 22:07:47 -07:00
Max Gekk 75dd86400c [SPARK-32908][SQL] Fix target error calculation in percentile_approx()
### What changes were proposed in this pull request?
1. Change the target error calculation according to the paper [Space-Efficient Online Computation of Quantile Summaries](http://infolab.stanford.edu/~datar/courses/cs361a/papers/quantiles.pdf). It says that the error `e = max(gi, deltai)/2` (see the page 59). Also this has clear explanation [ε-approximate quantiles](http://www.mathcs.emory.edu/~cheung/Courses/584/Syllabus/08-Quantile/Greenwald.html#proofprop1).
2. Added a test to check different accuracies.
3. Added an input CSV file `percentile_approx-input.csv.bz2` to the resource folder `sql/catalyst/src/main/resources` for the test.

### Why are the changes needed?
To fix incorrect percentile calculation, see an example in SPARK-32908.

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

### How was this patch tested?
- By running existing tests in `QuantileSummariesSuite` and in `ApproximatePercentileQuerySuite`.
- Added new test `SPARK-32908: maximum target error in percentile_approx` to `ApproximatePercentileQuerySuite`.

Closes #29784 from MaxGekk/fix-percentile_approx-2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-18 10:47:06 +09:00
Peter Toth 4ced58862c [SPARK-32635][SQL] Fix foldable propagation
### What changes were proposed in this pull request?
This PR rewrites `FoldablePropagation` rule to replace attribute references in a node with foldables coming only from the node's children.

Before this PR in the case of this example (with setting`spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation`):
```scala
val a = Seq("1").toDF("col1").withColumn("col2", lit("1"))
val b = Seq("2").toDF("col1").withColumn("col2", lit("2"))
val aub = a.union(b)
val c = aub.filter($"col1" === "2").cache()
val d = Seq("2").toDF( "col4")
val r = d.join(aub, $"col2" === $"col4").select("col4")
val l = c.select("col2")
val df = l.join(r, $"col2" === $"col4", "LeftOuter")
df.show()
```
foldable propagation happens incorrectly:
```
 Join LeftOuter, (col2#6 = col4#34)                                                              Join LeftOuter, (col2#6 = col4#34)
!:- Project [col2#6]                                                                             :- Project [1 AS col2#6]
 :  +- InMemoryRelation [col1#4, col2#6], StorageLevel(disk, memory, deserialized, 1 replicas)   :  +- InMemoryRelation [col1#4, col2#6], StorageLevel(disk, memory, deserialized, 1 replicas)
 :        +- Union                                                                               :        +- Union
 :           :- *(1) Project [value#1 AS col1#4, 1 AS col2#6]                                    :           :- *(1) Project [value#1 AS col1#4, 1 AS col2#6]
 :           :  +- *(1) Filter (isnotnull(value#1) AND (value#1 = 2))                            :           :  +- *(1) Filter (isnotnull(value#1) AND (value#1 = 2))
 :           :     +- *(1) LocalTableScan [value#1]                                              :           :     +- *(1) LocalTableScan [value#1]
 :           +- *(2) Project [value#10 AS col1#13, 2 AS col2#15]                                 :           +- *(2) Project [value#10 AS col1#13, 2 AS col2#15]
 :              +- *(2) Filter (isnotnull(value#10) AND (value#10 = 2))                          :              +- *(2) Filter (isnotnull(value#10) AND (value#10 = 2))
 :                 +- *(2) LocalTableScan [value#10]                                             :                 +- *(2) LocalTableScan [value#10]
 +- Project [col4#34]                                                                            +- Project [col4#34]
    +- Join Inner, (col2#6 = col4#34)                                                               +- Join Inner, (col2#6 = col4#34)
       :- Project [value#31 AS col4#34]                                                                :- Project [value#31 AS col4#34]
       :  +- LocalRelation [value#31]                                                                  :  +- LocalRelation [value#31]
       +- Project [col2#6]                                                                             +- Project [col2#6]
          +- Union false, false                                                                           +- Union false, false
             :- Project [1 AS col2#6]                                                                        :- Project [1 AS col2#6]
             :  +- LocalRelation [value#1]                                                                   :  +- LocalRelation [value#1]
             +- Project [2 AS col2#15]                                                                       +- Project [2 AS col2#15]
                +- LocalRelation [value#10]                                                                     +- LocalRelation [value#10]

```
and so the result is wrong:
```
+----+----+
|col2|col4|
+----+----+
|   1|null|
+----+----+
```

After this PR foldable propagation will not happen incorrectly and the result is correct:
```
+----+----+
|col2|col4|
+----+----+
|   2|   2|
+----+----+
```

### Why are the changes needed?
To fix a correctness issue.

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

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

Closes #29771 from peter-toth/SPARK-32635-fix-foldable-propagation.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-18 08:17:23 +09:00
Chao Sun 482a79a5e3 [SPARK-24994][SQL][FOLLOW-UP] Handle foldable, timezone and cleanup
### What changes were proposed in this pull request?

This is a follow-up on #29565, and addresses a few issues in the last PR:
- style issue pointed by [this comment](https://github.com/apache/spark/pull/29565#discussion_r487646749)
- skip optimization when `fromExp` is foldable (by [this comment](https://github.com/apache/spark/pull/29565#discussion_r487646973)) as there could be more efficient rule to apply for this case.
- pass timezone info to the generated cast on the literal value
- a bunch of cleanups and test improvements

Originally I plan to handle this when implementing [SPARK-32858](https://issues.apache.org/jira/browse/SPARK-32858) but now think it's better to isolate these changes from that.

### Why are the changes needed?

To fix a few left over issues in the above PR.

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

No

### How was this patch tested?

Added a test for the foldable case. Otherwise relying on existing tests.

Closes #29775 from sunchao/SPARK-24994-followup.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-17 07:50:39 -07:00
Liang-Chi Hsieh bd38e0be83 [SPARK-32903][SQL] GeneratePredicate should be able to eliminate common sub-expressions
### What changes were proposed in this pull request?

This patch proposes to make GeneratePredicate eliminate common sub-expressions.

### Why are the changes needed?

Both GenerateMutableProjection and GenerateUnsafeProjection, such codegen objects can eliminate common sub-expressions. But GeneratePredicate currently doesn't do it.

We encounter a customer issue that a Filter pushed down through a Project causes performance issue, compared with not pushed down case. The issue is one expression used in Filter predicates are run many times. Due to the complex schema, the query nodes are not wholestage codegen, so it runs Filter.doExecute and then call GeneratePredicate. The common expression was run many time and became performance bottleneck. GeneratePredicate should be able to eliminate common sub-expressions for such case.

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

No

### How was this patch tested?

Unit tests.

Closes #29776 from viirya/filter-pushdown.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-17 05:39:40 +00:00
Linhong Liu 40ef5c91ad [SPARK-32816][SQL] Fix analyzer bug when aggregating multiple distinct DECIMAL columns
### What changes were proposed in this pull request?
This PR fixes a conflict between `RewriteDistinctAggregates` and `DecimalAggregates`.
In some cases, `DecimalAggregates` will wrap the decimal column to `UnscaledValue` using
different rules for different aggregates.

This means, same distinct column with different aggregates will change to different distinct columns
after `DecimalAggregates`. For example:
`avg(distinct decimal_col), sum(distinct decimal_col)` may change to
`avg(distinct UnscaledValue(decimal_col)), sum(distinct decimal_col)`

We assume after `RewriteDistinctAggregates`, there will be at most one distinct column in aggregates,
but `DecimalAggregates` breaks this assumption. To fix this, we have to switch the order of these two
rules.

### Why are the changes needed?
bug fix

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

### How was this patch tested?
added test cases

Closes #29673 from linhongliu-db/SPARK-32816.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-16 16:53:25 +00:00
Yuming Wang 3bc13e6412 [SPARK-32706][SQL] Improve cast string to decimal type
### What changes were proposed in this pull request?

This pr makes cast string type to decimal decimal type fast fail if precision larger that 38.

### Why are the changes needed?

It is very slow if precision very large.

Benchmark and benchmark result:
```scala
import org.apache.spark.benchmark.Benchmark
val bd1 = new java.math.BigDecimal("6.0790316E+25569151")
val bd2 = new java.math.BigDecimal("6.0790316E+25");

val benchmark = new Benchmark("Benchmark string to decimal", 1, minNumIters = 2)
benchmark.addCase(bd1.toString) { _ =>
  println(Decimal(bd1).precision)
}
benchmark.addCase(bd2.toString) { _ =>
  println(Decimal(bd2).precision)
}
benchmark.run()
```
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.6
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Benchmark string to decimal:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
6.0790316E+25569151                                9340           9381          57          0.0  9340094625.0       1.0X
6.0790316E+25                                         0              0           0          0.5        2150.0 4344230.1X
```
Stacktrace:
![image](https://user-images.githubusercontent.com/5399861/92941705-4c868980-f483-11ea-8a15-b93acde8c0f4.png)

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

No.

### How was this patch tested?

Unit test and benchmark test:
Dataset | Before this pr (Seconds) | After this pr (Seconds)
-- | -- | --
https://issues.apache.org/jira/secure/attachment/13011406/part-00000.parquet | 2640 | 2

Closes #29731 from wangyum/SPARK-32706.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-16 14:08:59 +00:00
Tanel Kiis 6051755bfe [SPARK-32688][SQL][TEST] Add special values to LiteralGenerator for float and double
### What changes were proposed in this pull request?

The `LiteralGenerator` for float and double datatypes was supposed to yield special values (NaN, +-inf) among others, but the `Gen.chooseNum` method does not yield values that are outside the defined range. The `Gen.chooseNum` for a wide range of floats and doubles does not yield values in the "everyday" range as stated in https://github.com/typelevel/scalacheck/issues/113 .

There is an similar class `RandomDataGenerator` that is used in some other tests. Added `-0.0` and `-0.0f` as special values to there too.

These changes revealed an inconsistency with the equality check between `-0.0` and `0.0`.

### Why are the changes needed?

The `LiteralGenerator` is mostly used in the `checkConsistencyBetweenInterpretedAndCodegen` method in `MathExpressionsSuite`. This change would have caught the bug fixed in #29495 .

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

No

### How was this patch tested?

Locally reverted #29495 and verified that the existing test cases caught the bug.

Closes #29515 from tanelk/SPARK-32688.

Authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-16 12:13:15 +09:00
ulysses 888b343587 [SPARK-32827][SQL] Add spark.sql.maxMetadataStringLength config
### What changes were proposed in this pull request?

Add a new config `spark.sql.maxMetadataStringLength`. This config aims to limit metadata value length, e.g. file location.

### Why are the changes needed?

Some metadata have been abbreviated by `...` when I tried to add some test in `SQLQueryTestSuite`. We need to replace such value to `notIncludedMsg`. That caused we can't replace that like location value by `className` since the `className` has been abbreviated.

Here is a case:
```
CREATE table  explain_temp1 (key int, val int) USING PARQUET;

EXPLAIN EXTENDED SELECT sum(distinct val) FROM explain_temp1;

-- ignore parsed,analyzed,optimized
-- The output like
== Physical Plan ==
*HashAggregate(keys=[], functions=[sum(distinct cast(val#x as bigint)#xL)], output=[sum(DISTINCT val)#xL])
+- Exchange SinglePartition, true, [id=#x]
   +- *HashAggregate(keys=[], functions=[partial_sum(distinct cast(val#x as bigint)#xL)], output=[sum#xL])
      +- *HashAggregate(keys=[cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL])
         +- Exchange hashpartitioning(cast(val#x as bigint)#xL, 4), true, [id=#x]
            +- *HashAggregate(keys=[cast(val#x as bigint) AS cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL])
               +- *ColumnarToRow
                  +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/home/runner/work/spark/spark/sql/core/spark-warehouse/org.apache.spark.sq...], PartitionFilters: ...
```

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

No, a new config.

### How was this patch tested?

new test.

Closes #29688 from ulysses-you/SPARK-32827.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 14:11:30 +00:00
tanel.kiis@gmail.com 7a17158a4d [SPARK-32868][SQL] Add more order irrelevant aggregates to EliminateSorts
### What changes were proposed in this pull request?

Mark `BitAggregate` as order irrelevant in `EliminateSorts`.

### Why are the changes needed?

Performance improvements in some queries

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

No

### How was this patch tested?

Generalized an existing UT

Closes #29740 from tanelk/SPARK-32868.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-14 22:52:33 +09:00
Yuanjian Li 5e825482d7 [SPARK-32844][SQL] Make DataFrameReader.table take the specified options for datasource v1
### What changes were proposed in this pull request?
Make `DataFrameReader.table` take the specified options for datasource v1.

### Why are the changes needed?
Keep the same behavior of v1/v2 datasource, the v2 fix has been done in SPARK-32592.

### Does this PR introduce _any_ user-facing change?
Yes. The DataFrameReader.table will take the specified options. Also, if there are the same key and value exists in specified options and table properties, an exception will be thrown.

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

Closes #29712 from xuanyuanking/SPARK-32844.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-14 09:20:24 +00:00
Chao Sun 3d08084022 [SPARK-24994][SQL] Add UnwrapCastInBinaryComparison optimizer to simplify literal types
### What changes were proposed in this pull request?

Currently, in cases like the following:
```sql
SELECT * FROM t WHERE age < 40
```
where `age` is of short type, Spark won't be able to simplify this and can only generate filter `cast(age, int) < 40`. This won't get pushed down to datasources and therefore is not optimized.

This PR proposes a optimizer rule to improve this when the following constraints are satisfied:
 - input expression is binary comparisons when one side is a cast operation and another is a literal.
 - both the cast child expression and literal are of integral type (i.e., byte, short, int or long)

When this is true, it tries to do several optimizations to either simplify the expression or move the cast to the literal side, so
result filter for the above case becomes `age < cast(40 as smallint)`. This is better since the cast can be optimized away later and the filter can be pushed down to data sources.

This PR follows a similar effort in Presto (https://prestosql.io/blog/2019/05/21/optimizing-the-casts-away.html). Here we only handles integral types but plan to extend to other types as follow-ups.

### Why are the changes needed?

As mentioned in the previous section, when cast is not optimized, it cannot be pushed down to data sources which can lead
to unnecessary IO and therefore longer job time and waste of resources. This helps to improve that.

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

No.

### How was this patch tested?

Added unit tests for both the optimizer rule and filter pushdown on datasource level for both Orc and Parquet.

Closes #29565 from sunchao/SPARK-24994.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 21:34:35 -07:00
Karol Chmist 3be552ccc8 [SPARK-30090][SHELL] Adapt Spark REPL to Scala 2.13
### What changes were proposed in this pull request?

This is an attempt to adapt Spark REPL to Scala 2.13.

It is based on a [scala-2.13 branch](https://github.com/smarter/spark/tree/scala-2.13) made by smarter.

I had to set Scala version to 2.13 in some places, and to adapt some other modules, before I could start working on the REPL itself. These are separate commits on the branch that probably would be fixed beforehand, and thus dropped before the merge of this PR.

I couldn't find a way to run the initialization code with existing REPL classes in Scala 2.13.2, so I [modified REPL in Scala](e9cc0dd547) to make it work. With this modification I managed to run Spark Shell, along with the units tests passing, which is good news.

The bad news is that it requires an upstream change in Scala, which must be accepted first. I'd be happy to change it if someone points a way to do it differently. If not, I'd propose a PR in Scala to introduce `ILoop.internalReplAutorunCode`.

### Why are the changes needed?

REPL in Scala changed quite a lot, so current version of Spark REPL needed to be adapted.

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

In the previous version of `SparkILoop`, a lot of Scala's `ILoop` code was [overridden and duplicated](2bc7b75537) to make the welcome message a bit more pleasant. In this PR, the message is in a bit different order, but it's still acceptable IMHO.

Before this PR:
```
20/05/15 15:32:39 WARN Utils: Your hostname, hermes resolves to a loopback address: 127.0.1.1; using 192.168.1.28 instead (on interface enp0s31f6)
20/05/15 15:32:39 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/05/15 15:32:39 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
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).
20/05/15 15:32:45 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041.
Spark context Web UI available at http://192.168.1.28:4041
Spark context available as 'sc' (master = local[*], app id = local-1589549565502).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.1-SNAPSHOT
      /_/

Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
Type in expressions to have them evaluated.
Type :help for more information.

scala>
```

With this PR:
```
20/05/15 15:32:15 WARN Utils: Your hostname, hermes resolves to a loopback address: 127.0.1.1; using 192.168.1.28 instead (on interface enp0s31f6)
20/05/15 15:32:15 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/05/15 15:32:15 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
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).
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.13.2-20200422-211118-706ef1b (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
Type in expressions to have them evaluated.
Type :help for more information.
Spark context Web UI available at http://192.168.1.28:4040
Spark context available as 'sc' (master = local[*], app id = local-1589549541259).
Spark session available as 'spark'.

scala>
```

It seems that currently the welcoming message is still an improvement from [the original ticket](https://issues.apache.org/jira/browse/SPARK-24785), albeit in a different order. As a bonus, some fragile code duplication was removed.

### How was this patch tested?

Existing tests pass in `repl`module. The REPL runs in a terminal and the following code executed correctly:

```
scala> spark.range(1000 * 1000 * 1000).count()
val res0: Long = 1000000000
```

Closes #28545 from karolchmist/scala-2.13-repl.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-12 18:15:15 -05:00
yangjiang fe2ab255d1 [MINOR][SQL] Fix a typo at 'spark.sql.sources.fileCompressionFactor' error message in SQLConf
### What changes were proposed in this pull request?

 fix typo in SQLConf

### Why are the changes needed?

typo fix to increase readability

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

### How was this patch tested?

no test

Closes #29668 from Ted-Jiang/fix_annotate.

Authored-by: yangjiang <yangjiang@ebay.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-11 08:05:34 -05:00
gengjiaan a22871f50a [SPARK-32777][SQL] Aggregation support aggregate function with multiple foldable expressions
### What changes were proposed in this pull request?
Spark SQL exists a bug show below:

```
spark.sql(
  " SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2, 3)")
  .show()
+-----------------+--------------------+
|count(DISTINCT 2)|count(DISTINCT 2, 3)|
+-----------------+--------------------+
|                1|                   1|
+-----------------+--------------------+

spark.sql(
  " SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3, 2)")
  .show()
+-----------------+--------------------+
|count(DISTINCT 2)|count(DISTINCT 3, 2)|
+-----------------+--------------------+
|                1|                   0|
+-----------------+--------------------+
```
The first query is correct, but the second query is not.
The root reason is the second query rewrited by `RewriteDistinctAggregates` who expand the output but lost the 2.

### Why are the changes needed?
Fix a bug.
`SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3, 2)` should return `1, 1`

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

### How was this patch tested?
New UT

Closes #29626 from beliefer/support-multiple-foldable-distinct-expressions.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 11:25:32 +00:00
Kent Yao 5669b212ec [SPARK-32840][SQL] Invalid interval value can happen to be just adhesive with the unit
### What changes were proposed in this pull request?
In this PR, we add a checker for STRING form interval value ahead for parsing multiple units intervals and fail directly if the interval value contains alphabets to prevent correctness issues like `interval '1 day 2' day`=`3 days`.

### Why are the changes needed?

fix correctness issue

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

yes, in spark 3.0.0 `interval '1 day 2' day`=`3 days` but now we fail with ParseException
### How was this patch tested?

add a test.

Closes #29708 from yaooqinn/SPARK-32840.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 11:20:05 +00:00
Takeshi Yamamuro 7eb76d6988 [SPARK-32828][SQL] Cast from a derived user-defined type to a base type
### What changes were proposed in this pull request?

This PR intends to fix an existing bug below in `UserDefinedTypeSuite`;
```
[info] - SPARK-19311: UDFs disregard UDT type hierarchy (931 milliseconds)
16:22:35.936 WARN org.apache.spark.sql.catalyst.expressions.SafeProjection: Expr codegen error and falling back to interpreter mode
org.apache.spark.SparkException: Cannot cast org.apache.spark.sql.ExampleSubTypeUDT46b1771f to org.apache.spark.sql.ExampleBaseTypeUDT31e8d979.
	at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeCastFunction(Cast.scala:891)
	at org.apache.spark.sql.catalyst.expressions.CastBase.doGenCode(Cast.scala:852)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:147)
    ...
```

### Why are the changes needed?

bugfix

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

No.

### How was this patch tested?

Added unit tests.

Closes #29691 from maropu/FixUdtBug.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-10 19:19:26 +09:00
Jungtaek Lim (HeartSaVioR) db89b0e1b8 [SPARK-32831][SS] Refactor SupportsStreamingUpdate to represent actual meaning of the behavior
### What changes were proposed in this pull request?

This PR renames `SupportsStreamingUpdate` to `SupportsStreamingUpdateAsAppend` as the new interface name represents the actual behavior clearer. This PR also removes the `update()` method (so the interface is more likely a marker), as the implementations of `SupportsStreamingUpdateAsAppend` should support append mode by default, hence no need to trigger some flag on it.

### Why are the changes needed?

SupportsStreamingUpdate was intended to revive the functionality of Streaming update output mode for internal data sources, but despite the name, that interface isn't really used to do actual update on sink; all sinks are implementing this interface to do append, so strictly saying, it's just to support update as append. Renaming the interface would make it clear.

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

No, as the class is only for internal data sources.

### How was this patch tested?

Jenkins test will follow.

Closes #29693 from HeartSaVioR/SPARK-32831.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-09-10 15:33:18 +09:00
Bryan Cutler e0538bd38c [SPARK-32312][SQL][PYTHON][TEST-JAVA11] Upgrade Apache Arrow to version 1.0.1
### What changes were proposed in this pull request?

Upgrade Apache Arrow to version 1.0.1 for the Java dependency and increase minimum version of PyArrow to 1.0.0.

This release marks a transition to binary stability of the columnar format (which was already informally backward-compatible going back to December 2017) and a transition to Semantic Versioning for the Arrow software libraries. Also note that the Java arrow-memory artifact has been split to separate dependence on netty-buffer and allow users to select an allocator. Spark will continue to use `arrow-memory-netty` to maintain performance benefits.

Version 1.0.0 - 1.0.0 include the following selected fixes/improvements relevant to Spark users:

ARROW-9300 - [Java] Separate Netty Memory to its own module
ARROW-9272 - [C++][Python] Reduce complexity in python to arrow conversion
ARROW-9016 - [Java] Remove direct references to Netty/Unsafe Allocators
ARROW-8664 - [Java] Add skip null check to all Vector types
ARROW-8485 - [Integration][Java] Implement extension types integration
ARROW-8434 - [C++] Ipc RecordBatchFileReader deserializes the Schema multiple times
ARROW-8314 - [Python] Provide a method to select a subset of columns of a Table
ARROW-8230 - [Java] Move Netty memory manager into a separate module
ARROW-8229 - [Java] Move ArrowBuf into the Arrow package
ARROW-7955 - [Java] Support large buffer for file/stream IPC
ARROW-7831 - [Java] unnecessary buffer allocation when calling splitAndTransferTo on variable width vectors
ARROW-6111 - [Java] Support LargeVarChar and LargeBinary types and add integration test with C++
ARROW-6110 - [Java] Support LargeList Type and add integration test with C++
ARROW-5760 - [C++] Optimize Take implementation
ARROW-300 - [Format] Add body buffer compression option to IPC message protocol using LZ4 or ZSTD
ARROW-9098 - RecordBatch::ToStructArray cannot handle record batches with 0 column
ARROW-9066 - [Python] Raise correct error in isnull()
ARROW-9223 - [Python] Fix to_pandas() export for timestamps within structs
ARROW-9195 - [Java] Wrong usage of Unsafe.get from bytearray in ByteFunctionsHelper class
ARROW-7610 - [Java] Finish support for 64 bit int allocations
ARROW-8115 - [Python] Conversion when mixing NaT and datetime objects not working
ARROW-8392 - [Java] Fix overflow related corner cases for vector value comparison
ARROW-8537 - [C++] Performance regression from ARROW-8523
ARROW-8803 - [Java] Row count should be set before loading buffers in VectorLoader
ARROW-8911 - [C++] Slicing a ChunkedArray with zero chunks segfaults

View release notes here:
https://arrow.apache.org/release/1.0.1.html
https://arrow.apache.org/release/1.0.0.html

### Why are the changes needed?

Upgrade brings fixes, improvements and stability guarantees.

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

No

### How was this patch tested?

Existing tests with pyarrow 1.0.0 and 1.0.1

Closes #29686 from BryanCutler/arrow-upgrade-100-SPARK-32312.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-10 14:16:19 +09:00
Liang-Chi Hsieh add267c4de [SPARK-32819][SQL] ignoreNullability parameter should be effective recursively
### What changes were proposed in this pull request?

This patch proposes to check `ignoreNullability` parameter recursively in `equalsStructurally` method.

### Why are the changes needed?

`equalsStructurally` is used to check type equality. We can optionally ask to ignore nullability check. But the parameter `ignoreNullability` is not passed recursively down to nested types. So it produces weird error like:

```
data type mismatch: argument 3 requires array<array<string>> type, however ... is of array<array<string>> type.
```

when running the query `select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array( x ) ) )`.

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

Yes, fixed a bug when running user query.

### How was this patch tested?

Unit tests.

Closes #29698 from viirya/SPARK-32819.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 02:53:22 +00:00
Wenchen Fan f7995c576a Revert "[SPARK-32677][SQL] Load function resource before create"
This reverts commit 05fcf26b79.
2020-09-09 18:15:22 +00:00
yangjie01 fc10511d15 [SPARK-32755][SQL][FOLLOWUP] Ensure -- method of AttributeSet have same behavior under Scala 2.12 and 2.13
### What changes were proposed in this pull request?
 `--` method of `AttributeSet` behave differently under Scala 2.12 and 2.13 because `--` method of `LinkedHashSet` in Scala 2.13 can't maintains the insertion order.

This pr use a Scala 2.12 based code to ensure `--` method of AttributeSet have same behavior under Scala 2.12 and 2.13.

### Why are the changes needed?
The behavior of `AttributeSet`  needs to be compatible with Scala 2.12 and 2.13

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

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

Scala 2.13: Manual test sub-suites of `PlanStabilitySuite`

- **Before** :293 TESTS FAILED

- **After**:13 TESTS FAILED(The remaining failures are not associated with the current issue)

Closes #29689 from LuciferYang/SPARK-32755-FOLLOWUP.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-09 14:40:49 +00:00
yangjie01 513d51a2c5 [SPARK-32808][SQL] Fix some test cases of sql/core module in scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to partial resolve [SPARK-32808](https://issues.apache.org/jira/browse/SPARK-32808), total of 26 failed test cases were fixed, the related suite as follow:

- `StreamingAggregationSuite` related test cases (2 FAILED -> Pass)

- `GeneratorFunctionSuite` related test cases (2 FAILED -> Pass)

- `UDFSuite` related test cases (2 FAILED -> Pass)

- `SQLQueryTestSuite` related test cases (5 FAILED -> Pass)

- `WholeStageCodegenSuite` related test cases (1 FAILED -> Pass)

- `DataFrameSuite` related test cases (3 FAILED -> Pass)

- `OrcV1QuerySuite\OrcV2QuerySuite` related test cases (4 FAILED -> Pass)

- `ExpressionsSchemaSuite` related test cases (1 FAILED -> Pass)

- `DataFrameStatSuite` related test cases (1 FAILED -> Pass)

- `JsonV1Suite\JsonV2Suite\JsonLegacyTimeParserSuite` related test cases (6 FAILED -> Pass)

The main change of this pr as following:

- Fix Scala 2.13 compilation problems in   `ShuffleBlockFetcherIterator`  and `Analyzer`

- Specified `Seq` to `scala.collection.Seq` in `objects.scala` and `GenericArrayData` because internal use `Seq` maybe `mutable.ArraySeq` and not easy to call `.toSeq`

- Should specified `Seq` to `scala.collection.Seq`  when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but `Seq` is `immutable.Seq` in Scala 2.13

- Use a compatible way to let `+` and `-` method  of `Decimal` having the same behavior in Scala 2.12 and Scala 2.13

- Call `toList` in `RelationalGroupedDataset.toDF` method when `groupingExprs` is `Stream` type because `Stream` can't serialize in Scala 2.13

- Add a manual sort to `classFunsMap` in `ExpressionsSchemaSuite` because `Iterable.groupBy` in Scala 2.13 has different result with `TraversableLike.groupBy`  in Scala 2.12

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

Should specified `Seq` to `scala.collection.Seq`  when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but the `Seq` is `immutable.Seq` in Scala 2.13

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/core -Pscala-2.13 -am
mvn test -pl sql/core -Pscala-2.13
```

**Before**
```
Tests: succeeded 8166, failed 319, canceled 1, ignored 52, pending 0
*** 319 TESTS FAILED ***

```

**After**

```
Tests: succeeded 8204, failed 286, canceled 1, ignored 52, pending 0
*** 286 TESTS FAILED ***

```

Closes #29660 from LuciferYang/SPARK-32808.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-09 08:53:44 -05:00
Wenchen Fan 4144b6da52 [SPARK-32764][SQL] -0.0 should be equal to 0.0
### What changes were proposed in this pull request?

This is a Spark 3.0 regression introduced by https://github.com/apache/spark/pull/26761. We missed a corner case that `java.lang.Double.compare` treats 0.0 and -0.0 as different, which breaks SQL semantic.

This PR adds back the `OrderingUtil`, to provide custom compare methods that take care of 0.0 vs -0.0

### Why are the changes needed?

Fix a correctness bug.

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

Yes, now `SELECT  0.0 > -0.0` returns false correctly as Spark 2.x.

### How was this patch tested?

new tests

Closes #29647 from cloud-fan/float.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-07 20:43:43 -07:00
Wenchen Fan 117a6f135b [SPARK-32638][SQL][FOLLOWUP] Move the plan rewriting methods to QueryPlan
### What changes were proposed in this pull request?

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

It moves the plan rewriting methods from `Analyzer` to `QueryPlan`, so that it can work with `SparkPlan` as well. This PR also does an improvement to support a corner case (The attribute to be replace stays together with an unresolved attribute), and make it more general, so that `WidenSetOperationTypes` can rewrite the plan in one shot like before.

### Why are the changes needed?

Code cleanup and generalize.

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

no

### How was this patch tested?

existing test

Closes #29643 from cloud-fan/cleanup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-08 09:54:05 +09:00
manuzhang c43460cf82 [SPARK-32753][SQL] Only copy tags to node with no tags
### What changes were proposed in this pull request?
Only copy tags to node with no tags when transforming plans.

### Why are the changes needed?
cloud-fan [made a good point](https://github.com/apache/spark/pull/29593#discussion_r482013121) that it doesn't make sense to append tags to existing nodes when nodes are removed. That will cause such bugs as duplicate rows when deduplicating and repartitioning by the same column with AQE.

```
spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1")
val df = spark.sql("select id from v1 group by id distribute by id")
println(df.collect().toArray.mkString(","))
println(df.queryExecution.executedPlan)

// With AQE
[4],[0],[3],[2],[1],[7],[6],[8],[5],[9],[4],[0],[3],[2],[1],[7],[6],[8],[5],[9]
AdaptiveSparkPlan(isFinalPlan=true)
+- CustomShuffleReader local
   +- ShuffleQueryStage 0
      +- Exchange hashpartitioning(id#183L, 10), true
         +- *(3) HashAggregate(keys=[id#183L], functions=[], output=[id#183L])
            +- Union
               :- *(1) Range (0, 10, step=1, splits=2)
               +- *(2) Range (0, 10, step=1, splits=2)

// Without AQE
[4],[7],[0],[6],[8],[3],[2],[5],[1],[9]
*(4) HashAggregate(keys=[id#206L], functions=[], output=[id#206L])
+- Exchange hashpartitioning(id#206L, 10), true
   +- *(3) HashAggregate(keys=[id#206L], functions=[], output=[id#206L])
      +- Union
         :- *(1) Range (0, 10, step=1, splits=2)
         +- *(2) Range (0, 10, step=1, splits=2)
```

It's too expensive to detect node removal so we make a compromise only to copy tags to node with no tags.

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

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

Closes #29593 from manuzhang/spark-32753.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 16:08:57 +00:00
ulysses 05fcf26b79 [SPARK-32677][SQL] Load function resource before create
### What changes were proposed in this pull request?

Change `CreateFunctionCommand` code that add class check before create function.

### Why are the changes needed?

We have different behavior between create permanent function and temporary function when function class is invaild. e.g.,
```
create function f as 'test.non.exists.udf';
-- Time taken: 0.104 seconds

create temporary function f as 'test.non.exists.udf'
-- Error in query: Can not load class 'test.non.exists.udf' when registering the function 'f', please make sure it is on the classpath;
```

And Hive also fails both of them.

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

Yes, user will get exception when create a invalid udf.

### How was this patch tested?

New test.

Closes #29502 from ulysses-you/function.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 06:00:23 +00:00
Kent Yao de44e9cfa0 [SPARK-32785][SQL] Interval with dangling parts should not results null
### What changes were proposed in this pull request?

bugfix for incomplete interval values, e.g. interval '1', interval '1 day 2', currently these cases will result null, but actually we should fail them with IllegalArgumentsException

### Why are the changes needed?

correctness

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

yes, incomplete intervals will throw exception now

#### before
```
bin/spark-sql -S -e "select interval '1', interval '+', interval '1 day -'"

NULL NULL NULL
```
#### after

```
-- !query
select interval '1'
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

Cannot parse the INTERVAL value: 1(line 1, pos 7)

== SQL ==
select interval '1'
```

### How was this patch tested?

unit tests added

Closes #29635 from yaooqinn/SPARK-32785.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 05:11:30 +00:00
Ali Afroozeh f55694638d [SPARK-32800][SQL] Remove ExpressionSet from the 2.13 branch
### What changes were proposed in this pull request?
This PR is a followup on #29598 and removes the `ExpressionSet` class from the 2.13 branch.

### Why are the changes needed?
`ExpressionSet` does not extend Scala `Set` anymore and this class is no longer needed in the 2.13 branch.

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

### How was this patch tested?
Passes existing tests

Closes #29648 from dbaliafroozeh/RemoveExpressionSetFrom2.13Branch.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-06 09:44:07 +09:00
Takeshi Yamamuro a6114d8fb8 [SPARK-32638][SQL] Corrects references when adding aliases in WidenSetOperationTypes
### What changes were proposed in this pull request?

This PR intends to fix a bug where references can be missing when adding aliases to widen data types in `WidenSetOperationTypes`. For example,
```
CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v);
SELECT t.v FROM (
  SELECT v FROM t3
  UNION ALL
  SELECT v + v AS v FROM t3
) t;

org.apache.spark.sql.AnalysisException: Resolved attribute(s) v#1 missing from v#3 in operator !Project [v#1]. Attribute(s) with the same name appear in the operation: v. Please check if the right attribute(s) are used.;;
!Project [v#1]  <------ the reference got missing
+- SubqueryAlias t
   +- Union
      :- Project [cast(v#1 as decimal(11,0)) AS v#3]
      :  +- Project [v#1]
      :     +- SubqueryAlias t3
      :        +- SubqueryAlias tbl
      :           +- LocalRelation [v#1]
      +- Project [v#2]
         +- Project [CheckOverflow((promote_precision(cast(v#1 as decimal(11,0))) + promote_precision(cast(v#1 as decimal(11,0)))), DecimalType(11,0), true) AS v#2]
            +- SubqueryAlias t3
               +- SubqueryAlias tbl
                  +- LocalRelation [v#1]
```
In the case, `WidenSetOperationTypes` added the alias `cast(v#1 as decimal(11,0)) AS v#3`, then the reference in the top `Project` got missing. This PR correct the reference (`exprId` and widen `dataType`) after adding aliases in the rule.

### Why are the changes needed?

bugfixes

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

No

### How was this patch tested?

Added unit tests

Closes #29485 from maropu/SPARK-32638.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-03 14:48:26 +00:00
Ali Afroozeh 0a6043f683 [SPARK-32755][SQL] Maintain the order of expressions in AttributeSet and ExpressionSet
### What changes were proposed in this pull request?
This PR changes `AttributeSet` and `ExpressionSet` to maintain the insertion order of the elements. More specifically, we:
- change the underlying data structure of `AttributeSet` from `HashSet` to `LinkedHashSet` to maintain the insertion order.
- `ExpressionSet` already uses a list to keep track of the expressions, however, since it is extending Scala's immutable.Set class, operations such as map and flatMap are delegated to the immutable.Set itself. This means that the result of these operations is not an instance of ExpressionSet anymore, rather it's a implementation picked up by the parent class. We also remove this inheritance from `immutable.Set `and implement the needed methods directly. ExpressionSet has a very specific semantics and it does not make sense to extend `immutable.Set` anyway.
- change the `PlanStabilitySuite` to not sort the attributes, to be able to catch changes in the order of expressions in different runs.

### Why are the changes needed?
Expressions identity is based on the `ExprId` which is an auto-incremented number. This means that the same query can yield a query plan with different expression ids in different runs. `AttributeSet` and `ExpressionSet` internally use a `HashSet` as the underlying data structure, and therefore cannot guarantee the a fixed order of operations in different runs. This can be problematic in cases we like to check for plan changes in different runs.

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

### How was this patch tested?
Passes `PlanStabilitySuite` after regenerating the golden files.

Closes #29598 from dbaliafroozeh/FixOrderOfExpressions.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-09-03 13:56:03 +02:00
Yuanjian Li 95f1e9549b [SPARK-32782][SS] Refactor StreamingRelationV2 and move it to catalyst
### What changes were proposed in this pull request?
Move StreamingRelationV2 to the catalyst module and bind with the Table interface.

### Why are the changes needed?
Currently, the StreamingRelationV2 is bind with TableProvider. Since the V2 relation is not bound with `DataSource`, to make it more flexible and have better expansibility, it should be moved to the catalyst module and bound with the Table interface. We did a similar thing for DataSourceV2Relation.

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

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

Closes #29633 from xuanyuanking/SPARK-32782.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-03 16:04:36 +09:00
Kent Yao 1fba286407 [SPARK-32781][SQL] Non-ASCII characters are mistakenly omitted in the middle of intervals
### What changes were proposed in this pull request?

This PR fails the interval values parsing when they contain non-ASCII characters which are silently omitted right now.

e.g. the case below should be invalid

```
select interval 'interval中文 1 day'
```

### Why are the changes needed?

bugfix, intervals should fail when containing invalid characters

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

yes,

#### before

select interval 'interval中文 1 day'  results 1 day, now it fails with

```
org.apache.spark.sql.catalyst.parser.ParseException

Cannot parse the INTERVAL value: interval中文 1 day
```

### How was this patch tested?

new tests

Closes #29632 from yaooqinn/SPARK-32781.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-03 04:56:40 +00:00
angerszhu 5e6173ebef [SPARK-31670][SQL] Trim unnecessary Struct field alias in Aggregate/GroupingSets
### What changes were proposed in this pull request?
Struct field both in GROUP BY and Aggregate Expresison with CUBE/ROLLUP/GROUPING SET will failed when analysis.

```
test("SPARK-31670") {
  withTable("t1") {
      sql(
        """
          |CREATE TEMPORARY VIEW t(a, b, c) AS
          |SELECT * FROM VALUES
          |('A', 1, NAMED_STRUCT('row_id', 1, 'json_string', '{"i": 1}')),
          |('A', 2, NAMED_STRUCT('row_id', 2, 'json_string', '{"i": 1}')),
          |('A', 2, NAMED_STRUCT('row_id', 2, 'json_string', '{"i": 2}')),
          |('B', 1, NAMED_STRUCT('row_id', 3, 'json_string', '{"i": 1}')),
          |('C', 3, NAMED_STRUCT('row_id', 4, 'json_string', '{"i": 1}'))
        """.stripMargin)

      checkAnswer(
        sql(
          """
            |SELECT a, c.json_string, SUM(b)
            |FROM t
            |GROUP BY a, c.json_string
            |WITH CUBE
            |""".stripMargin),
        Row("A", "{\"i\": 1}", 3) :: Row("A", "{\"i\": 2}", 2) :: Row("A", null, 5) ::
          Row("B", "{\"i\": 1}", 1) :: Row("B", null, 1) ::
          Row("C", "{\"i\": 1}", 3) :: Row("C", null, 3) ::
          Row(null, "{\"i\": 1}", 7) :: Row(null, "{\"i\": 2}", 2) :: Row(null, null, 9) :: Nil)

  }
}
```
Error 
```
[info] - SPARK-31670 *** FAILED *** (2 seconds, 857 milliseconds)
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: expression 't.`c`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
[info]   Aggregate [a#247, json_string#248, spark_grouping_id#246L], [a#247, c#223.json_string AS json_string#241, sum(cast(b#222 as bigint)) AS sum(b)#243L]
[info]   +- Expand [List(a#221, b#222, c#223, a#244, json_string#245, 0), List(a#221, b#222, c#223, a#244, null, 1), List(a#221, b#222, c#223, null, json_string#245, 2), List(a#221, b#222, c#223, null, null, 3)], [a#221, b#222, c#223, a#247, json_string#248, spark_grouping_id#246L]
[info]      +- Project [a#221, b#222, c#223, a#221 AS a#244, c#223.json_string AS json_string#245]
[info]         +- SubqueryAlias t
[info]            +- Project [col1#218 AS a#221, col2#219 AS b#222, col3#220 AS c#223]
[info]               +- Project [col1#218, col2#219, col3#220]
[info]                  +- LocalRelation [col1#218, col2#219, col3#220]
[info]
```
For Struct type Field, when we resolve it, it will construct with Alias. When struct field in GROUP BY with CUBE/ROLLUP etc,  struct field in groupByExpression and aggregateExpression will be resolved with different exprId as below
```
'Aggregate [cube(a#221, c#223.json_string AS json_string#240)], [a#221, c#223.json_string AS json_string#241, sum(cast(b#222 as bigint)) AS sum(b)#243L]
+- SubqueryAlias t
   +- Project [col1#218 AS a#221, col2#219 AS b#222, col3#220 AS c#223]
      +- Project [col1#218, col2#219, col3#220]
         +- LocalRelation [col1#218, col2#219, col3#220]
```
This makes `ResolveGroupingAnalytics.constructAggregateExprs()` failed to replace aggreagteExpression use expand groupByExpression attribute since there exprId is not same. then error happened.

### Why are the changes needed?
Fix analyze bug

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

### How was this patch tested?
Added UT

Closes #28490 from AngersZhuuuu/SPARK-31670.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-02 13:49:09 +00:00
Karol Chmist 7511e43c50 [SPARK-32756][SQL] Fix CaseInsensitiveMap usage for Scala 2.13
### What changes were proposed in this pull request?

This is a follow-up of #29160. This allows Spark SQL project to compile for Scala 2.13.

### Why are the changes needed?

It's needed for #28545

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

No

### How was this patch tested?

I compiled with Scala 2.13. It fails in `Spark REPL` project, which will be fixed by #28545

Closes #29584 from karolchmist/SPARK-32364-scala-2.13.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-02 08:27:00 -05:00
liwensun f0851e95c6 [SPARK-32776][SS] Limit in streaming should not be optimized away by PropagateEmptyRelation
### What changes were proposed in this pull request?

PropagateEmptyRelation will not be applied to LIMIT operators in streaming queries.

### Why are the changes needed?

Right now, the limit operator in a streaming query may get optimized away when the relation is empty. This can be problematic for stateful streaming, as this empty batch will not write any state store files, and the next batch will fail when trying to read these state store files and throw a file not found error.

We should not let PropagateEmptyRelation optimize away the Limit operator for streaming queries.

This PR is intended as a small and safe fix for PropagateEmptyRelation. A fundamental fix that can prevent this from happening again in the future and in other optimizer rules is more desirable, but that's a much larger task.

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

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

Closes #29623 from liwensun/spark-32776.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 18:05:06 +09:00
Zhenhua Wang 2a88a20271 [SPARK-32754][SQL][TEST] Unify to assertEqualJoinPlans for join reorder suites
### What changes were proposed in this pull request?

Now three join reorder suites(`JoinReorderSuite`, `StarJoinReorderSuite`, `StarJoinCostBasedReorderSuite`) all contain an `assertEqualPlans` method and the logic is almost the same. We can extract the method to a single place for code simplicity.

### Why are the changes needed?

To reduce code redundancy.

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #29594 from wzhfy/unify_assertEqualPlans_joinReorder.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-01 09:08:35 -07:00
Kris Mok 6e5bc39e17 [SPARK-32624][SQL][FOLLOWUP] Fix regression in CodegenContext.addReferenceObj on nested Scala types
### What changes were proposed in this pull request?

Use `CodeGenerator.typeName()` instead of `Class.getCanonicalName()` in `CodegenContext.addReferenceObj()` for getting the runtime class name for an object.

### Why are the changes needed?

https://github.com/apache/spark/pull/29439 fixed a bug in `CodegenContext.addReferenceObj()` for `Array[Byte]` (i.e. Spark SQL's `BinaryType`) objects, but unfortunately it introduced a regression for some nested Scala types.

For example, for `implicitly[Ordering[UTF8String]]`, after that PR `CodegenContext.addReferenceObj()` would return `((null) references[0] /* ... */)`. The actual type for `implicitly[Ordering[UTF8String]]` is `scala.math.LowPriorityOrderingImplicits$$anon$3` in Scala 2.12.10, and `Class.getCanonicalName()` returns `null` for that class.

On the other hand, `Class.getName()` is safe to use for all non-array types, and Janino will happily accept the type name returned from `Class.getName()` for nested types. `CodeGenerator.typeName()` happens to do the right thing by correctly handling arrays and otherwise use `Class.getName()`. So it's a better alternative than `Class.getCanonicalName()`.

Side note: rule of thumb for using Java reflection in Spark: it may be tempting to use `Class.getCanonicalName()`, but for functions that may need to handle Scala types, please avoid it due to potential issues with nested Scala types.
Instead, use `Class.getName()` or utility functions in `org.apache.spark.util.Utils` (e.g. `Utils.getSimpleName()` or `Utils.getFormattedClassName()` etc).

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

No.

### How was this patch tested?

Added new unit test case for the regression case in `CodeGenerationSuite`.

Closes #29602 from rednaxelafx/spark-32624-followup.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-01 15:15:11 +09:00
Chao Sun 94d313b061 [SPARK-32721][SQL][FOLLOWUP] Simplify if clauses with null and boolean
### What changes were proposed in this pull request?

This is a follow-up on SPARK-32721 and PR #29567. In the previous PR we missed two more cases that can be optimized:
```
if(p, false, null) ==> and(not(p), null)
if(p, true, null) ==> or(p, null)
```

### Why are the changes needed?

By transforming if to boolean conjunctions or disjunctions, we can enable more filter pushdown to datasources.

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

No

### How was this patch tested?

Added unit tests.

Closes #29603 from sunchao/SPARK-32721-2.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-09-01 06:06:25 +00:00
Chao Sun 1453a09a63 [SPARK-32721][SQL] Simplify if clauses with null and boolean
### What changes were proposed in this pull request?

The following if clause:
```sql
if(p, null, false)
```
can be simplified to:
```sql
and(p, null)
```
Similarly, the clause:
```sql
if(p, null, true)
```
can be simplified to
```sql
or(not(p), null)
```
iff the predicate `p` is non-nullable, i.e., can be evaluated to either true or false, but not null.

### Why are the changes needed?

Converting if to or/and clauses can better push filters down.

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

No.

### How was this patch tested?

Unit tests.

Closes #29567 from sunchao/SPARK-32721.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-08-31 20:59:54 +00:00
Huaxin Gao 806140de40 [SPARK-32592][SQL] Make DataFrameReader.table take the specified options
### What changes were proposed in this pull request?
pass specified options in DataFrameReader.table to JDBCTableCatalog.loadTable

### Why are the changes needed?
Currently, `DataFrameReader.table` ignores the specified options. The options specified like the following are lost.
```
    val df = spark.read
      .option("partitionColumn", "id")
      .option("lowerBound", "0")
      .option("upperBound", "3")
      .option("numPartitions", "2")
      .table("h2.test.people")
```
We need to make `DataFrameReader.table` take the specified options.

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

### How was this patch tested?
Manually test for now. Will add a test after V2 JDBC read is implemented.

Closes #29535 from huaxingao/table_options.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-31 13:21:15 +00:00
Kent Yao 6dacba7fa0 [SPARK-32733][SQL] Add extended information - arguments/examples/since/notes of expressions to the remarks field of GetFunctionsOperation
### What changes were proposed in this pull request?

This PR adds extended information of a function including arguments, examples, notes and the since field to the SparkGetFunctionOperation

### Why are the changes needed?

better user experience, it will help JDBC users to have a better understanding of our builtin functions

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

Yes, BI tools and JDBC users will get full information on a spark function instead of only fragmentary usage info.

e.g. date_part

#### before

```
date_part(field, source) - Extracts a part of the date/timestamp or interval source.
```
#### after

```
    Usage:
      date_part(field, source) - Extracts a part of the date/timestamp or interval source.

    Arguments:
      * field - selects which part of the source should be extracted, and supported string values are as same as the fields of the equivalent function `EXTRACT`.
      * source - a date/timestamp or interval column from where `field` should be extracted

    Examples:
      > SELECT date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456');
       2019
      > SELECT date_part('week', timestamp'2019-08-12 01:00:00.123456');
       33
      > SELECT date_part('doy', DATE'2019-08-12');
       224
      > SELECT date_part('SECONDS', timestamp'2019-10-01 00:00:01.000001');
       1.000001
      > SELECT date_part('days', interval 1 year 10 months 5 days);
       5
      > SELECT date_part('seconds', interval 5 hours 30 seconds 1 milliseconds 1 microseconds);
       30.001001

    Note:
      The date_part function is equivalent to the SQL-standard function `EXTRACT(field FROM source)`

    Since: 3.0.0

```

### How was this patch tested?

New tests

Closes #29577 from yaooqinn/SPARK-32733.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-31 11:03:01 +09:00
Udbhav30 065f17386d [SPARK-32481][CORE][SQL] Support truncate table to move data to trash
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

### How was this patch tested?
new UTs added

Closes #29552 from Udbhav30/truncate.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-30 10:25:32 -07:00
Wenchen Fan ccc0250a08 [SPARK-32718][SQL] Remove unnecessary keywords for interval units
### What changes were proposed in this pull request?

Remove the YEAR, MONTH, DAY, HOUR, MINUTE, SECOND keywords. They are not useful in the parser, as we need to support plural like YEARS, so the parser has to accept the general identifier as interval unit anyway.

### Why are the changes needed?

These keywords are reserved in ANSI. If Spark has these keywords, then they become reserved under ANSI mode. This makes Spark not able to run TPCDS queries as they use YEAR as alias name.

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

No

### How was this patch tested?

Added `TPCDSQueryANSISuite`, to make sure Spark with ANSI mode can run TPCDS queries.

Closes #29560 from cloud-fan/keyword.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-29 14:06:01 -07:00
Takeshi Yamamuro 0cb91b8c18 [SPARK-32704][SQL] Logging plan changes for execution
### What changes were proposed in this pull request?

Since we only log plan changes for analyzer/optimizer now, this PR intends to add code to log plan changes in the preparation phase in `QueryExecution` for execution.
```
scala> spark.sql("SET spark.sql.optimizer.planChangeLog.level=WARN")
scala> spark.range(10).groupBy("id").count().queryExecution.executedPlan
...
20/08/26 09:32:36 WARN PlanChangeLogger:
=== Applying Rule org.apache.spark.sql.execution.CollapseCodegenStages ===
!HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])              *(1) HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])
!+- HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])   +- *(1) HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])
!   +- Range (0, 10, step=1, splits=4)                                                          +- *(1) Range (0, 10, step=1, splits=4)

20/08/26 09:32:36 WARN PlanChangeLogger:
=== Result of Batch Preparations ===
!HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])              *(1) HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])
!+- HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])   +- *(1) HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])
!   +- Range (0, 10, step=1, splits=4)                                                          +- *(1) Range (0, 10, step=1, splits=4)
```

### Why are the changes needed?

Easy debugging for executed plans

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

No.

### How was this patch tested?

Added unit tests.

Closes #29544 from maropu/PlanLoggingInPreparations.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-28 16:35:47 +00:00
Kent Yao 0626901bcb [SPARK-32729][SQL][DOCS] Add missing since version for math functions
### What changes were proposed in this pull request?

Add missing since version for math functions, including
SPARK-8223 shiftright/shiftleft
SPARK-8215 pi
SPARK-8212 e
SPARK-6829 sin/asin/sinh/cos/acos/cosh/tan/atan/tanh/ceil/floor/rint/cbrt/signum/isignum/Fsignum/Lsignum/degrees/radians/log/log10/log1p/exp/expm1/pow/hypot/atan2
SPARK-8209 conv
SPARK-8213 factorial
SPARK-20751 cot
SPARK-2813 sqrt
SPARK-8227 unhex
SPARK-8218 log(a,b)
SPARK-8207 bin
SPARK-8214 hex
SPARK-8206 round
SPARK-14614 bround

### Why are the changes needed?

fix SQL docs
### Does this PR introduce _any_ user-facing change?

yes, doc updated

### How was this patch tested?

passing doc generation.

Closes #29571 from yaooqinn/minor.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-29 00:30:31 +09:00
yi.wu c3b9404253 [SPARK-32717][SQL] Add a AQEOptimizer for AdaptiveSparkPlanExec
### What changes were proposed in this pull request?

This PR proposes to add a specific `AQEOptimizer` for the `AdaptiveSparkPlanExec` instead of implementing an anonymous `RuleExecutor`. At the same time, this PR also adds the configuration `spark.sql.adaptive.optimizer.excludedRules`, which follows the same pattern of `Optimizer`, to make the `AQEOptimizer` more flexible for users and developers.

### Why are the changes needed?

Currently, `AdaptiveSparkPlanExec` has implemented an anonymous `RuleExecutor` to apply the AQE optimize rules on the plan. However, the anonymous class usually could be inconvenient to maintain and extend for the long term.

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

No.

### How was this patch tested?

It's a pure refactor so pass existing tests should be ok.

Closes #29559 from Ngone51/impro-aqe-optimizer.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 21:23:53 +09:00
HyukjinKwon c154629171 [SPARK-32183][DOCS][PYTHON] User Guide - PySpark Usage Guide for Pandas with Apache Arrow
### What changes were proposed in this pull request?

This PR proposes to move Arrow usage guide from Spark documentation site to PySpark documentation site (at "User Guide").

Here is the demo for reviewing quicker: https://hyukjin-spark.readthedocs.io/en/stable/user_guide/arrow_pandas.html

### Why are the changes needed?

To have a single place for PySpark users, and better documentation.

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

Yes, it will move https://spark.apache.org/docs/latest/sql-pyspark-pandas-with-arrow.html to our PySpark documentation.

### How was this patch tested?

```bash
cd docs
SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 jekyll serve --watch
```

and

```bash
cd python/docs
make clean html
```

Closes #29548 from HyukjinKwon/SPARK-32183.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 15:09:06 +09:00
Terry Kim baaa756dee [SPARK-32516][SQL][FOLLOWUP] 'path' option cannot coexist with path parameter for DataFrameWriter.save(), DataStreamReader.load() and DataStreamWriter.start()
### What changes were proposed in this pull request?

This is a follow up PR to #29328 to apply the same constraint where `path` option cannot coexist with path parameter to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`.

### Why are the changes needed?

The current behavior silently overwrites the `path` option if path parameter is passed to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`.

For example,
```
Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2")
```
will write the result to `/tmp/path2`.

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

Yes, if `path` option coexists with path parameter to any of the above methods, it will throw `AnalysisException`:
```
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2")
org.apache.spark.sql.AnalysisException: There is a 'path' option set and save() is called with a  path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set 'spark.sql.legacy.pathOptionBehavior.enabled' to 'true'.;
```

The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.

### How was this patch tested?

Added new tests.

Closes #29543 from imback82/path_option.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 06:21:04 +00:00
Dongjoon Hyun 2dee4352a0 Revert "[SPARK-32481][CORE][SQL] Support truncate table to move data to trash"
This reverts commit 5c077f0580.
2020-08-26 11:24:35 -07:00
Udbhav30 5c077f0580 [SPARK-32481][CORE][SQL] Support truncate table to move data to trash
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

### How was this patch tested?
new UTs added

Closes #29387 from Udbhav30/tuncateTrash.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-25 23:38:43 -07:00
Sean Owen a9d4e60a90 [SPARK-32614][SQL] Don't apply comment processing if 'comment' unset for CSV
### What changes were proposed in this pull request?

Spark's CSV source can optionally ignore lines starting with a comment char. Some code paths check to see if it's set before applying comment logic (i.e. not set to default of `\0`), but many do not, including the one that passes the option to Univocity. This means that rows beginning with a null char were being treated as comments even when 'disabled'.

### Why are the changes needed?

To avoid dropping rows that start with a null char when this is not requested or intended. See JIRA for an example.

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

Nothing beyond the effect of the bug fix.

### How was this patch tested?

Existing tests plus new test case.

Closes #29516 from srowen/SPARK-32614.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-26 00:25:58 +09:00
fqaiser94@gmail.com 3f1e56d4ca [SPARK-32641][SQL] withField + getField should return null if original struct was null
### What changes were proposed in this pull request?

There is a bug in the way the optimizer rule in `SimplifyExtractValueOps` is currently written in master branch which yields incorrect results in scenarios like the following:
```
sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
.select($"struct_col".withField("d", lit(4)).getField("d").as("d"))

// currently returns this:
+---+
|d  |
+---+
|4  |
+---+

// when in fact it should return this:
+----+
|d   |
+----+
|null|
+----+
```
The changes in this PR will fix this bug.

### Why are the changes needed?

To fix the aforementioned bug. Optimizer rules should improve the performance of the  query but yield exactly the same results.

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

Yes, this bug will no longer occur.
That said, this isn't something to be concerned about as this bug was introduced in Spark 3.1 and Spark 3.1 has yet to be released.

### How was this patch tested?

Unit tests were added. Jenkins must pass them.

Closes #29522 from fqaiser94/SPARK-32641.

Authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-25 04:59:37 +00:00
Yesheng Ma 3eee915b47 [MINOR][SQL] Add missing documentation for LongType mapping
### What changes were proposed in this pull request?

Added Java docs for Long data types in the Row class.

### Why are the changes needed?

The Long datatype is somehow missing in Row.scala's `apply` and `get` methods.

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

No

### How was this patch tested?

Existing UTs.

Closes #29534 from yeshengm/docs-fix.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 11:20:01 +09:00
yangjie01 a30bb0cfda [SPARK-32550][SQL][FOLLOWUP] Eliminate negative impact on HyperLogLogSuite
### What changes were proposed in this pull request?
Change to use `dataTypes.foreach` instead of get the element use specified index in `def this(dataTypes: Seq[DataType]) `constructor of `SpecificInternalRow` because the random access performance is unsatisfactory if the input argument not a `IndexSeq`.

This pr followed srowen's  advice.

### Why are the changes needed?
I found that SPARK-32550 had some negative impact on performance, the typical cases is "deterministic cardinality estimation" in `HyperLogLogPlusPlusSuite` when rsd is 0.001, we found the code that is significantly slower is line 41 in `HyperLogLogPlusPlusSuite`: `new SpecificInternalRow(hll.aggBufferAttributes.map(_.dataType)) `

08b951b1cb/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala (L40-L44)

The size of "hll.aggBufferAttributes" in this case is 209716, the results of comparison before and after spark-32550 merged are as follows, The unit is ns:

  | After   SPARK-32550 createBuffer | After   SPARK-32550 end to end | Before   SPARK-32550 createBuffer | Before   SPARK-32550 end to end
-- | -- | -- | -- | --
rsd 0.001, n   1000 | 52715513243 | 53004810687 | 195807999 | 773977677
rsd 0.001, n   5000 | 51881246165 | 52519358215 | 13689949 | 249974855
rsd 0.001, n   10000 | 52234282788 | 52374639172 | 14199071 | 183452846
rsd 0.001, n   50000 | 55503517122 | 55664035449 | 15219394 | 584477125
rsd 0.001, n   100000 | 51862662845 | 52116774177 | 19662834 | 166483678
rsd 0.001, n   500000 | 51619226715 | 52183189526 | 178048012 | 16681330
rsd 0.001, n   1000000 | 54861366981 | 54976399142 | 226178708 | 18826340
rsd 0.001, n   5000000 | 52023602143 | 52354615149 | 388173579 | 15446409
rsd 0.001, n   10000000 | 53008591660 | 53601392304 | 533454460 | 16033032

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

### How was this patch tested?
`mvn test -pl sql/catalyst -DwildcardSuites=org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlusSuite -Dtest=none`

**Before**:

```
Run completed in 8 minutes, 18 seconds.
Total number of tests run: 5
Suites: completed 2, aborted 0
Tests: succeeded 5, failed 0, canceled 0, ignored 0, pending 0
```

**After**
```
Run completed in 7 seconds, 65 milliseconds.
Total number of tests run: 5
Suites: completed 2, aborted 0
Tests: succeeded 5, failed 0, canceled 0, ignored 0, pending 0
```

Closes #29529 from LuciferYang/revert-spark-32550.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 11:13:01 +09:00
Terry Kim e3a88a9767 [SPARK-32516][SQL] 'path' option cannot coexist with load()'s path parameters
### What changes were proposed in this pull request?

This PR proposes to make the behavior consistent for the `path` option when loading dataframes with a single path (e.g, `option("path", path).format("parquet").load(path)` vs. `option("path", path).parquet(path)`) by disallowing `path` option to coexist with `load`'s path parameters.

### Why are the changes needed?

The current behavior is inconsistent:
```scala
scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test")

scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show
+-----+
|value|
+-----+
|    1|
+-----+

scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show
+-----+
|value|
+-----+
|    1|
|    1|
+-----+
```

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

Yes, now if the `path` option is specified along with `load`'s path parameters, it would fail:
```scala
scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test")

scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show
org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.;
  at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:232)
  ... 47 elided

scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show
org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.;
  at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:250)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:778)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:756)
  ... 47 elided
```

The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.

### How was this patch tested?

Added a test

Closes #29328 from imback82/dfw_option.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-24 16:30:30 +00:00
yangjie01 25c7d0fe6a [SPARK-32526][SQL] Pass all test of sql/catalyst module in Scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to resolve [SPARK-32526](https://issues.apache.org/jira/browse/SPARK-32526), all remaining failed cases are fixed.

The main change of this pr as follow:

- Change of `ExecutorAllocationManager.scala` for core module compilation in Scala 2.13, it's a blocking problem

- Change `Seq[_]` to `scala.collection.Seq[_]` refer to failed cases

- Added different expected plan of `Test 4: Star with several branches` of StarJoinCostBasedReorderSuite  for Scala 2.13 because the candidates plans:

```
Join Inner, (d1_pk#5 = f1_fk1#0)
:- Join Inner, (f1_fk2#1 = d2_pk#8)
:  :- Join Inner, (f1_fk3#2 = d3_pk#11)
```
and

```
Join Inner, (f1_fk2#1 = d2_pk#8)
:- Join Inner, (d1_pk#5 = f1_fk1#0)
:  :- Join Inner, (f1_fk3#2 = d3_pk#11)
```

have same cost `Cost(200,9200)`, but `HashMap` is rewritten in scala 2.13 and The order of iterations leads to different results.

This pr fix test cases as follow:

- LiteralExpressionSuite (1 FAILED -> PASS)
- StarJoinCostBasedReorderSuite ( 1 FAILED-> PASS)
- ObjectExpressionsSuite( 2 FAILED-> PASS)
- ScalaReflectionSuite (1 FAILED-> PASS)
- RowEncoderSuite (10 FAILED-> PASS)
- ExpressionEncoderSuite  (ABORTED-> PASS)

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

### How was this patch tested?
<!--
- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/catalyst -Pscala-2.13 -am
mvn test -pl sql/catalyst -Pscala-2.13
```

**Before**
```
Tests: succeeded 4035, failed 17, canceled 0, ignored 6, pending 0
*** 1 SUITE ABORTED ***
*** 15 TESTS FAILED ***
```

**After**

```
Tests: succeeded 4338, failed 0, canceled 0, ignored 6, pending 0
All tests passed.
```

Closes #29434 from LuciferYang/sql-catalyst-tests.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-08-22 09:24:16 -05:00
Brandon Jiang 1450b5e095 [MINOR][DOCS] fix typo for docs,log message and comments
### What changes were proposed in this pull request?
Fix typo for docs, log messages and comments

### Why are the changes needed?
typo fix to increase readability

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

### How was this patch tested?
manual test has been performed to test the updated

Closes #29443 from brandonJY/spell-fix-doc.

Authored-by: Brandon Jiang <Brandon.jiang.a@outlook.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-22 06:45:35 +09:00
Wenchen Fan 3dca81e4f5 [SPARK-32669][SQL][TEST] Expression unit tests should explore all cases that can lead to null result
### What changes were proposed in this pull request?

Add document to `ExpressionEvalHelper`, and ask people to explore all the cases that can lead to null results (including null in struct fields, array elements and map values).

This PR also fixes `ComplexTypeSuite.GetArrayStructFields` to explore all the null cases.

### Why are the changes needed?

It happened several times that we hit correctness bugs caused by wrong expression nullability. When writing unit tests, we usually don't test the nullability flag directly, and it's too late to add such tests for all expressions.

In https://github.com/apache/spark/pull/22375, we extended the expression test framework, which checks the nullability flag when the expected result/field/element is null.

This requires the test cases to explore all the cases that can lead to null results

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

no

### How was this patch tested?

I reverted 5d296ed39e locally, and `ComplexTypeSuite` can catch the bug.

Closes #29493 from cloud-fan/small.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-22 06:23:46 +09:00
Wenchen Fan 8b119f1663 [SPARK-32640][SQL] Downgrade Janino to fix a correctness bug
### What changes were proposed in this pull request?

This PR reverts https://github.com/apache/spark/pull/27860 to downgrade Janino, as the new version has a bug.

### Why are the changes needed?

The symptom is about NaN comparison. For code below
```
if (double_value <= 0.0) {
  ...
} else {
  ...
}
```

If `double_value` is NaN, `NaN <= 0.0` is false and we should go to the else branch. However, current Spark goes to the if branch and causes correctness issues like SPARK-32640.

One way to fix it is:
```
boolean cond = double_value <= 0.0;
if (cond) {
  ...
} else {
  ...
}
```

I'm not familiar with Janino so I don't know what's going on there.

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

Yes, fix correctness bugs.

### How was this patch tested?

a new test

Closes #29495 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-20 13:26:39 -07:00
Yuming Wang 409fea30cc [SPARK-32624][SQL] Use getCanonicalName to fix byte[] compile issue
### What changes were proposed in this pull request?
```scala
scala> Array[Byte](1, 2).getClass.getName
res13: String = [B

scala> Array[Byte](1, 2).getClass.getCanonicalName
res14: String = byte[]
```

This pr replace `getClass.getName` with `getClass.getCanonicalName` in `CodegenContext.addReferenceObj` to fix `byte[]` compile issue:
```
...
/* 030 */       value_1 = org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary(value_2, (([B) references[0] /* min */)) >= 0 && org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary(value_2, (([B) references[1] /* max */)) <= 0;
/* 031 */     }
/* 032 */     return !isNull_1 && value_1;
/* 033 */   }
/* 034 */
/* 035 */
/* 036 */ }

20:49:54.886 WARN org.apache.spark.sql.catalyst.expressions.Predicate: Expr codegen error and falling back to interpreter mode
java.util.concurrent.ExecutionException: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 30, Column 81: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 30, Column 81: Unexpected token "[" in primary
...
```

### Why are the changes needed?

Fix compile issue when compiling generated code.

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

No.

### How was this patch tested?

Unit test.

Closes #29439 from wangyum/SPARK-32624.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-08-19 05:20:26 -07:00
angerszhu 03e2de99ab [SPARK-32608][SQL] Script Transform ROW FORMAT DELIMIT value should format value
### What changes were proposed in this pull request?
For SQL
```
SELECT TRANSFORM(a, b, c)
  ROW FORMAT DELIMITED
  FIELDS TERMINATED BY ','
  LINES TERMINATED BY '\n'
  NULL DEFINED AS 'null'
  USING 'cat' AS (a, b, c)
  ROW FORMAT DELIMITED
  FIELDS TERMINATED BY ','
  LINES TERMINATED BY '\n'
  NULL DEFINED AS 'NULL'
FROM testData
```
The correct

TOK_TABLEROWFORMATFIELD should be `, `nut actually ` ','`

TOK_TABLEROWFORMATLINES should be `\n`  but actually` '\n'`

### Why are the changes needed?
Fix string value format

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

### How was this patch tested?
Added UT

Closes #29428 from AngersZhuuuu/SPARK-32608.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-19 08:31:58 +00:00
Samir Khan e15ae60a53 [SPARK-32550][SQL] Make SpecificInternalRow constructors faster by using while loops instead of maps
### What changes were proposed in this pull request?
Change maps in two constructors of SpecificInternalRow to while loops.

### Why are the changes needed?
This was originally noticed with https://github.com/apache/spark/pull/29353 and https://github.com/apache/spark/pull/29354 and will have impacts on performance of reading ORC and Avro files. Ran AvroReadBenchmarks with the new cases of nested and array'd structs in https://github.com/apache/spark/pull/29352. Haven't run benchmarks for ORC but can do that if needed.

**Before:**
```
Nested Struct Scan:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Nested Struct                                     74674          75319         912          0.0      142429.1       1.0X

Array of Struct Scan:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Array of Structs                                  34193          34339         206          0.0       65217.9       1.0X
```
**After:**
```
Nested Struct Scan:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Nested Struct                                     48451          48619         237          0.0       92413.2       1.0X

Array of Struct Scan:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Array of Structs                                  18518          18683         234          0.0       35319.6       1.0X
```

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

### How was this patch tested?
Ran AvroReadBenchmarks with the new cases of nested and array'd structs in https://github.com/apache/spark/pull/29352.

Closes #29366 from msamirkhan/spark-32550.

Lead-authored-by: Samir Khan <muhammad.samir.khan@gmail.com>
Co-authored-by: skhan04 <samirkhan@verizonmedia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-19 14:57:34 +09:00
Wenchen Fan f33b64a656 [SPARK-32652][SQL] ObjectSerializerPruning fails for RowEncoder
### What changes were proposed in this pull request?

Update `ObjectSerializerPruning.alignNullTypeInIf`, to consider the isNull check generated in `RowEncoder`, which is `Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil)`.

### Why are the changes needed?

Query fails if we don't fix this bug, due to type mismatch in `If`.

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

Yes, the failed query can run after this fix.

### How was this patch tested?

new tests

Closes #29467 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-19 13:50:29 +09:00
Wenchen Fan b94c67b502 Revert "[SPARK-32511][SQL] Add dropFields method to Column class"
This reverts commit 0c850c71e7.
2020-08-17 13:18:46 +08:00
Cheng Su 8f0fef1843 [SPARK-32399][SQL] Full outer shuffled hash join
### What changes were proposed in this pull request?

Add support for full outer join inside shuffled hash join. Currently if the query is a full outer join, we only use sort merge join as the physical operator. However it can be CPU and IO intensive in case input table is large for sort merge join. Shuffled hash join on the other hand saves the sort CPU and IO compared to sort merge join, especially when table is large.

This PR implements the full outer join as followed:
* Process rows from stream side by looking up hash relation, and mark the matched rows from build side by:
  * for joining with unique key, a `BitSet` is used to record matched rows from build side (`key index` to represent each row)
  * for joining with non-unique key, a `HashSet[Long]` is  used to record matched rows from build side (`key index` + `value index` to represent each row).
`key index` is defined as the index into key addressing array `longArray` in `BytesToBytesMap`.
`value index` is defined as the iterator index of values for same key.

* Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`)

For context, this PR was originally implemented as followed (up to commit e3322766d4):
1. Construct hash relation from build side, with extra boolean value at the end of row to track look up information (done in `ShuffledHashJoinExec.buildHashedRelation` and `UnsafeHashedRelation.apply`).
2. Process rows from stream side by looking up hash relation, and mark the matched rows from build side be looked up (done in `ShuffledHashJoinExec.fullOuterJoin`).
3. Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`).

See discussion of pros and cons between these two approaches [here](https://github.com/apache/spark/pull/29342#issuecomment-672275450), [here](https://github.com/apache/spark/pull/29342#issuecomment-672288194) and [here](https://github.com/apache/spark/pull/29342#issuecomment-672640531).

TODO: codegen for full outer shuffled hash join can be implemented in another followup PR.

### Why are the changes needed?

As implementation in this PR, full outer shuffled hash join will have overhead to iterate build side twice (once for building hash map, and another for outputting non-matching rows), and iterate stream side once. However, full outer sort merge join needs to iterate both sides twice, and sort the large table can be more CPU and IO intensive. So full outer shuffled hash join can be more efficient than sort merge join when stream side is much more larger than build side.

For example query below, full outer SHJ saved 30% wall clock time compared to full outer SMJ.

```
def shuffleHashJoin(): Unit = {
    val N: Long = 4 << 22
    withSQLConf(
      SQLConf.SHUFFLE_PARTITIONS.key -> "2",
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000") {
      codegenBenchmark("shuffle hash join", N) {
        val df1 = spark.range(N).selectExpr(s"cast(id as string) as k1")
        val df2 = spark.range(N / 10).selectExpr(s"cast(id * 10 as string) as k2")
        val df = df1.join(df2, col("k1") === col("k2"), "full_outer")
        df.noop()
    }
  }
}
```

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join off
  Stopped after 2 iterations, 16602 ms
  Running case: shuffle hash join on
  Stopped after 5 iterations, 31911 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join off                              7900           8301         567          2.1         470.9       1.0X
shuffle hash join on                               6250           6382          95          2.7         372.5       1.3X
```

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

No.

### How was this patch tested?

Added unit test in `JoinSuite.scala`, `AbstractBytesToBytesMapSuite.java` and `HashedRelationSuite.scala`.

Closes #29342 from c21/full-outer-shj.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-17 08:06:19 +09:00
Yuming Wang c280c7f529 [SPARK-32625][SQL] Log error message when falling back to interpreter mode
### What changes were proposed in this pull request?

This pr log the error message when falling back to interpreter mode.

### Why are the changes needed?

Not all error messages are in `CodeGenerator`, such as:
```
21:48:44.612 WARN org.apache.spark.sql.catalyst.expressions.Predicate: Expr codegen error and falling back to interpreter mode
java.lang.IllegalArgumentException: Can not interpolate org.apache.spark.sql.types.Decimal into code block.
	at org.apache.spark.sql.catalyst.expressions.codegen.Block$BlockHelper$.$anonfun$code$1(javaCode.scala:240)
	at org.apache.spark.sql.catalyst.expressions.codegen.Block$BlockHelper$.$anonfun$code$1$adapted(javaCode.scala:236)
	at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
	at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
```

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

No.

### How was this patch tested?

Manual test.

Closes #29440 from wangyum/SPARK-32625.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-15 12:31:32 -07:00
yi.wu c6be2074cc [SPARK-32616][SQL] Window operators should be added determinedly
### What changes were proposed in this pull request?

Use the `LinkedHashMap` instead of `immutable.Map` to hold the `Window` expressions in `ExtractWindowExpressions.addWindow`.

### Why are the changes needed?

This is a bug fix for https://github.com/apache/spark/pull/29270. In that PR, the generated plan(especially for the queries q47, q49, q57) on Jenkins always can not match the golden plan generated on my laptop.

It happens because `ExtractWindowExpressions.addWindow` now uses `immutable.Map` to hold the `Window` expressions by the key `(spec.partitionSpec, spec.orderSpec, WindowFunctionType.functionType(expr))` and converts the map to `Seq` at the end. Then, the `Seq` is used to add Window operators on top of the child plan. However, for the same query, the order of Windows expression inside the `Seq` could be undetermined when the expression id changes(which can affect the key). As a result, the same query could have different plans because of the undetermined order of Window operators.

Therefore, we use `LinkedHashMap`, which records the insertion order of entries, to make the adding order determined.

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

Maybe yes, users now always see the same plan for the same queries with multiple Window operators.

### How was this patch tested?

It's really hard to make a reproduce demo. I just tested manually with https://github.com/apache/spark/pull/29270 and it looks good.

Closes #29432 from Ngone51/fix-addWindow.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-14 13:29:48 +00:00
alexander-daskalov 10edeafc69 [MINOR][SQL] Fixed approx_count_distinct rsd param description
### What changes were proposed in this pull request?

In the docs concerning the approx_count_distinct I have changed the description of the rsd parameter from **_maximum estimation error allowed_** to _**maximum relative standard deviation allowed**_

### Why are the changes needed?

Maximum estimation error allowed can be misleading. You can set the target relative standard deviation, which affects the estimation error, but on given runs the estimation error can still be above the rsd parameter.

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

This PR should make it easier for users reading the docs to understand that the rsd parameter in approx_count_distinct doesn't cap the estimation error, but just sets the target deviation instead,

### How was this patch tested?

No tests, as no code changes were made.

Closes #29424 from Comonut/fix-approx_count_distinct-rsd-param-description.

Authored-by: alexander-daskalov <alexander.daskalov@adevinta.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-14 22:10:41 +09:00
yangjie01 6ae2cb2db3 [SPARK-32526][SQL] Fix some test cases of sql/catalyst module in scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to partial resolve [SPARK-32526](https://issues.apache.org/jira/browse/SPARK-32526), total of 88 failed and 2 aborted test cases were fixed, the related suite as follow:

- `DataSourceV2AnalysisBaseSuite` related test cases (71 FAILED -> Pass)
- `TreeNodeSuite` (1 FAILED -> Pass)
- `MetadataSuite `(1 FAILED -> Pass)
- `InferFiltersFromConstraintsSuite `(3 FAILED -> Pass)
- `StringExpressionsSuite ` (1 FAILED -> Pass)
- `JacksonParserSuite ` (1 FAILED -> Pass)
- `HigherOrderFunctionsSuite `(1 FAILED -> Pass)
- `ExpressionParserSuite` (1 FAILED -> Pass)
- `CollectionExpressionsSuite `(6 FAILED -> Pass)
- `SchemaUtilsSuite` (2 FAILED -> Pass)
- `ExpressionSetSuite `(ABORTED -> Pass)
- `ArrayDataIndexedSeqSuite `(ABORTED -> Pass)

The main change of this pr as following:

- `Optimizer` and `Analyzer` are changed to pass compile, `ArrayBuffer` is not a `Seq` in scala 2.13, call `toSeq` method manually to compatible with Scala 2.12

- `m.mapValues().view.force` pattern return a `Map` in scala 2.12 but return a `IndexedSeq` in scala 2.13, call `toMap` method manually to compatible with Scala 2.12. `TreeNode` are changed to pass `DataSourceV2AnalysisBaseSuite` related test cases and `TreeNodeSuite` failed case.

- call `toMap` method of `Metadata#hash` method `case map` branch because `map.mapValues` return `Map` in Scala 2.12 and return `MapView` in Scala 2.13.

- `impl` contact method of `ExpressionSet` in Scala 2.13 version refer to `ExpressionSet` in Scala 2.12 to support `+ + ` method conform to `ExpressionSet` semantics

- `GenericArrayData` not accept `ArrayBuffer` input, call `toSeq` when use `ArrayBuffer` construction `GenericArrayData`   for Scala version compatibility

-  Call `toSeq` in `RandomDataGenerator#randomRow` method to ensure contents of `fields` is `Seq` not `ArrayBuffer`

-  Call `toSeq` Let `JacksonParser#parse` still return a `Seq` because the check method of `JacksonParserSuite#"skipping rows using pushdown filters"` dependence on `Seq` type
- Call `toSeq` in `AstBuilder#visitFunctionCall`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13

- Add a `LongType` match to `ArraySetLike.nullValueHolder`

- Add a `sorted` to ensure `duplicateColumns` string in `SchemaUtils.checkColumnNameDuplication` method error message have a deterministic order

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/catalyst -Pscala-2.13 -am
mvn test -pl sql/catalyst -Pscala-2.13
```

**Before**
```
Tests: succeeded 3853, failed 103, canceled 0, ignored 6, pending 0
*** 3 SUITES ABORTED ***
*** 103 TESTS FAILED ***
```

**After**

```
Tests: succeeded 4035, failed 17, canceled 0, ignored 6, pending 0
*** 1 SUITE ABORTED ***
*** 15 TESTS FAILED ***
```

Closes #29370 from LuciferYang/fix-DataSourceV2AnalysisBaseSuite.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-08-13 11:46:30 -05:00
fqaiser94@gmail.com 0c850c71e7 [SPARK-32511][SQL] Add dropFields method to Column class
### What changes were proposed in this pull request?

Added a new `dropFields` method to the `Column` class.
This method should allow users to drop a `StructField` in a `StructType` column (with similar semantics to the `drop` method on `Dataset`).

### Why are the changes needed?

Often Spark users have to work with deeply nested data e.g. to fix a data quality issue with an existing `StructField`. To do this with the existing Spark APIs, users have to rebuild the entire struct column.

For example, let's say you have the following deeply nested data structure which has a data quality issue (`5` is missing):
```
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

val data = spark.createDataFrame(sc.parallelize(
      Seq(Row(Row(Row(1, 2, 3), Row(Row(4, null, 6), Row(7, 8, 9), Row(10, 11, 12)), Row(13, 14, 15))))),
      StructType(Seq(
        StructField("a", StructType(Seq(
          StructField("a", StructType(Seq(
            StructField("a", IntegerType),
            StructField("b", IntegerType),
            StructField("c", IntegerType)))),
          StructField("b", StructType(Seq(
            StructField("a", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType)))),
            StructField("b", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType)))),
            StructField("c", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType))))
          ))),
          StructField("c", StructType(Seq(
            StructField("a", IntegerType),
            StructField("b", IntegerType),
            StructField("c", IntegerType))))
        )))))).cache

data.show(false)
+---------------------------------+
|a                                |
+---------------------------------+
|[[1, 2, 3], [[4,, 6], [7, 8, 9]]]|
+---------------------------------+
```
Currently, to drop the missing value users would have to do something like this:
```
val result = data.withColumn("a",
  struct(
    $"a.a",
    struct(
      struct(
        $"a.b.a.a",
        $"a.b.a.c"
      ).as("a"),
      $"a.b.b",
      $"a.b.c"
    ).as("b"),
    $"a.c"
  ))

result.show(false)
+---------------------------------------------------------------+
|a                                                              |
+---------------------------------------------------------------+
|[[1, 2, 3], [[4, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]|
+---------------------------------------------------------------+
```
As you can see above, with the existing methods users must call the `struct` function and list all fields, including fields they don't want to change. This is not ideal as:
>this leads to complex, fragile code that cannot survive schema evolution.
[SPARK-16483](https://issues.apache.org/jira/browse/SPARK-16483)

In contrast, with the method added in this PR, a user could simply do something like this to get the same result:
```
val result = data.withColumn("a", 'a.dropFields("b.a.b"))
result.show(false)
+---------------------------------------------------------------+
|a                                                              |
+---------------------------------------------------------------+
|[[1, 2, 3], [[4, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]|
+---------------------------------------------------------------+

```

This is the second of maybe 3 methods that could be added to the `Column` class to make it easier to manipulate nested data.
Other methods under discussion in [SPARK-22231](https://issues.apache.org/jira/browse/SPARK-22231) include `withFieldRenamed`.
However, this should be added in a separate PR.

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

Only one minor change. If the user submits the following query:
```
df.withColumn("a", $"a".withField(null, null))
```
instead of throwing:
```
java.lang.IllegalArgumentException: requirement failed: fieldName cannot be null
```
it will now throw:
```
java.lang.IllegalArgumentException: requirement failed: col cannot be null
```
I don't believe its should be an issue to change this because:
- neither message is incorrect
- Spark 3.1.0 has yet to be released

but please feel free to correct me if I am wrong.

### How was this patch tested?

New unit tests were added. Jenkins must pass them.

### Related JIRAs:
More discussion on this topic can be found here:
- https://issues.apache.org/jira/browse/SPARK-22231
- https://issues.apache.org/jira/browse/SPARK-16483

Closes #29322 from fqaiser94/SPARK-32511.

Lead-authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Co-authored-by: fqaiser94 <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-13 03:28:25 +00:00
stczwd 60fa8e304d [SPARK-31694][SQL] Add SupportsPartitions APIs on DataSourceV2
### What changes were proposed in this pull request?
There are no partition Commands, such as AlterTableAddPartition supported in DatasourceV2, it is widely used in mysql or hive or other datasources. Thus it is necessary to defined Partition API to support these Commands.

We defined the partition API as part of Table API, as it will change table data sometimes. And a partition is composed of identifier and properties, while identifier is defined with InternalRow and properties is defined as a Map.

### Does this PR introduce _any_ user-facing change?
Yes. This PR will enable user to use some partition commands

### How was this patch tested?
run all tests and add some partition api tests

Closes #28617 from stczwd/SPARK-31694.

Authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-12 17:25:47 +00:00
Yuming Wang 5d130f0360 [SPARK-32586][SQL] Fix NumberFormatException error message when ansi is enabled
### What changes were proposed in this pull request?

This pr fixes the error message of `NumberFormatException` when casting invalid input to FractionalType and enabling **ansi**:
```
spark-sql> set spark.sql.ansi.enabled=true;
spark.sql.ansi.enabled	true
spark-sql> create table SPARK_32586 using parquet as select 's' s;
spark-sql> select * from SPARK_32586 where s > 1.13D;
java.lang.NumberFormatException: invalid input syntax for type numeric: columnartorow_value_0
```

After this pr:
```
spark-sql> select * from SPARK_32586 where s > 1.13D;
java.lang.NumberFormatException: invalid input syntax for type numeric: s
```

### Why are the changes needed?

Improve error message.

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

No.

### How was this patch tested?

Unit test.

Closes #29405 from wangyum/SPARK-32586.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 13:16:57 +09:00
gengjiaan e7c1204f6c [SPARK-32540][SQL] Eliminate the filter clause in aggregate
### What changes were proposed in this pull request?
Spark SQL supported filter clause in aggregate, for example:
`select sum(distinct id) filter (where sex = 'man') from student;`
But sometimes we can eliminate the filter clause in aggregate.
`SELECT COUNT(DISTINCT 1) FILTER (WHERE true) FROM testData;`
could be transformed to
`SELECT COUNT(DISTINCT 1) FROM testData;`
`SELECT COUNT(DISTINCT 1) FILTER (WHERE false) FROM testData;`
could be transformed to
`SELECT 0 FROM testData;`

### Why are the changes needed?
Optimize the filter clause in aggregation

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

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

Closes #29369 from beliefer/eliminate-filter-clause.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 16:20:19 +00:00
xuewei.linxuewei c37357a092 [SPARK-32573][SQL] Anti Join Improvement with EmptyHashedRelation and EmptyHashedRelationWithAllNullKeys
### What changes were proposed in this pull request?
In [SPARK-32290](https://issues.apache.org/jira/browse/SPARK-32290), we introduced several new types of HashedRelation.

* EmptyHashedRelation
* EmptyHashedRelationWithAllNullKeys

They were all limited to used only in NAAJ scenario. These new HashedRelation could be applied to other scenario for performance improvements.

* EmptyHashedRelation could also be used in Normal AntiJoin for fast stop
* While AQE is on and buildSide is EmptyHashedRelationWithAllNullKeys, can convert NAAJ to a Empty LocalRelation to skip meaningless data iteration since in Single-Key NAAJ, if null key exists in BuildSide, will drop all records in streamedSide.

This Patch including two changes.

* using EmptyHashedRelation to do fast stop for common anti join as well
* In AQE, eliminate BroadcastHashJoin(NAAJ) if buildSide is a EmptyHashedRelationWithAllNullKeys

### Why are the changes needed?
LeftAntiJoin could apply `fast stop` when BuildSide is EmptyHashedRelation, While within AQE with EmptyHashedRelationWithAllNullKeys, we can eliminate the NAAJ. This should be a performance improvement in AntiJoin.

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

### How was this patch tested?

* added case in AdaptiveQueryExecSuite.
* added case in HashedRelationSuite.
* Make sure SubquerySuite JoinSuite SQLQueryTestSuite passed.

Closes #29389 from leanken/leanken-SPARK-32573.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 06:23:51 +00:00
allisonwang-db 1b7443bd9a [SPARK-32216][SQL] Remove redundant ProjectExec
### What changes were proposed in this pull request?
This PR added a physical rule to remove redundant project nodes. A `ProjectExec` is redundant when
1. It has the same output attributes and order as its child's output when ordering of these attributes is required.
2. It has the same output attributes as its child's output when attribute output ordering is not required.

For example:
After Filter:
```
== Physical Plan ==
*(1) Project [a#14L, b#15L, c#16, key#17]
+- *(1) Filter (isnotnull(a#14L) AND (a#14L > 5))
   +- *(1) ColumnarToRow
      +- FileScan parquet [a#14L,b#15L,c#16,key#17]
```
The `Project a#14L, b#15L, c#16, key#17` is redundant because its output is exactly the same as filter's output.

Before Aggregate:
```
== Physical Plan ==
*(2) HashAggregate(keys=[key#17], functions=[sum(a#14L), last(b#15L, false)], output=[sum_a#39L, key#17, last_b#41L])
+- Exchange hashpartitioning(key#17, 5), true, [id=#77]
   +- *(1) HashAggregate(keys=[key#17], functions=[partial_sum(a#14L), partial_last(b#15L, false)], output=[key#17, sum#49L, last#50L, valueSet#51])
      +- *(1) Project [key#17, a#14L, b#15L]
         +- *(1) Filter (isnotnull(a#14L) AND (a#14L > 100))
            +- *(1) ColumnarToRow
               +- FileScan parquet [a#14L,b#15L,key#17]
```
The `Project key#17, a#14L, b#15L` is redundant because hash aggregate doesn't require child plan's output to be in a specific order.

### Why are the changes needed?

It removes unnecessary query nodes and makes query plan cleaner.

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

No

### How was this patch tested?

Unit tests

Closes #29031 from allisonwang-db/remove-project.

Lead-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Co-authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 03:14:15 +00:00
Yuanjian Li b03761e330 [SPARK-32456][SS] Check the Distinct by assuming it as Aggregate for Structured Streaming
### What changes were proposed in this pull request?
Check the Distinct nodes by assuming it as Aggregate in `UnsupportOperationChecker` for streaming.

### Why are the changes needed?
We want to fix 2 things here:

1. Give better error message for Distinct related operations in append mode that doesn't have a watermark

We use the union streams as the example, distinct in SQL has the same issue. Since the union clause in SQL has the requirement of deduplication, the parser will generate `Distinct(Union)` and the optimizer rule `ReplaceDistinctWithAggregate` will change it to `Aggregate(Union)`. This logic is of both batch and streaming queries. However, in the streaming, the aggregation will be wrapped by state store operations so we need extra checking logic in `UnsupportOperationChecker`.

Before this change, the SS union queries in Append mode will get the following confusing error when the watermark is lacking.
```
java.util.NoSuchElementException: None.get
	at scala.None$.get(Option.scala:529)
	at scala.None$.get(Option.scala:527)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec.$anonfun$doExecute$9(statefulOperators.scala:346)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.timeTakenMs(Utils.scala:561)
	at org.apache.spark.sql.execution.streaming.StateStoreWriter.timeTakenMs(statefulOperators.scala:112)
...
```

2. Make `Distinct` in complete mode runnable.

Before this fix, the distinct in complete mode will throw the exception:
```
Complete output mode not supported when there are no streaming aggregations on streaming DataFrames/Datasets;
```

### Does this PR introduce _any_ user-facing change?
Yes, return a better error message.

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

Closes #29256 from xuanyuanking/SPARK-32456.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-10 14:01:31 +09:00
allisonwang-db 924c161544 [SPARK-32337][SQL] Show initial plan in AQE plan tree string
### What changes were proposed in this pull request?
This PR adds initial plan in `AdaptiveSparkPlanExec` and generates tree string for both current plan and initial plan. When the adaptive plan is not final, `Current Plan` will be used to indicate current physical plan, and `Final Plan` will be used when the adaptive plan is final. The difference between `Current Plan` and `Final Plan` here is that current plan indicates an intermediate state. The plan is subject to further transformations, while final plan represents an end state, which means the plan will no longer be changed.

Examples:

Before this change:
```
AdaptiveSparkPlan isFinalPlan=true
+- *(3) BroadcastHashJoin
   :- BroadcastQueryStage 2
       ...
```
`EXPLAIN FORMATTED`
```
== Physical Plan ==
AdaptiveSparkPlan (9)
+- BroadcastHashJoin Inner BuildRight (8)
   :- Project (3)
   :  +- Filter (2)
```

After this change
```
AdaptiveSparkPlan isFinalPlan=true
+- == Final Plan ==
   *(3) BroadcastHashJoin
   :- BroadcastQueryStage 2
   :  +- BroadcastExchange
           ...
+- == Initial Plan ==
   SortMergeJoin
   :- Sort
   :  +- Exchange
           ...
```

`EXPLAIN FORMATTED`
```
== Physical Plan ==
AdaptiveSparkPlan (9)
+- == Current Plan ==
   BroadcastHashJoin Inner BuildRight (8)
   :- Project (3)
   :  +- Filter (2)
+- == Initial Plan ==
   BroadcastHashJoin Inner BuildRight (8)
   :- Project (3)
   :  +- Filter (2)
```

### Why are the changes needed?
It provides better visibility into the plan change introduced by AQE.

### Does this PR introduce _any_ user-facing change?
Yes. It changed the AQE plan output string.

### How was this patch tested?
Unit test

Closes #29137 from allisonwang-db/aqe-plan.

Lead-authored-by: allisonwang-db <allison.wang@databricks.com>
Co-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-10 04:49:37 +00:00
Liang-Chi Hsieh 7b6e1d5cec [SPARK-25557][SQL] Nested column predicate pushdown for ORC
### What changes were proposed in this pull request?

We added nested column predicate pushdown for Parquet in #27728. This patch extends the feature support to ORC.

### Why are the changes needed?

Extending the feature to ORC for feature parity. Better performance for handling nested predicate pushdown.

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

No

### How was this patch tested?

Unit tests.

Closes #28761 from viirya/SPARK-25557.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-07 08:07:41 -07:00
Wenchen Fan d5682c13a2 [SPARK-32528][SQL][TEST] The analyze method should make sure the plan is analyzed
### What changes were proposed in this pull request?

This PR updates the `analyze` method to make sure the plan can be resolved. It also fixes some miswritten optimizer tests.

### Why are the changes needed?

It's error-prone if the `analyze` method can return an unresolved plan.

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

no

### How was this patch tested?

test only

Closes #29349 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-07 07:36:08 +00:00
Max Gekk 3a437ed22b [SPARK-32501][SQL] Convert null to "null" in structs, maps and arrays while casting to strings
### What changes were proposed in this pull request?
Convert `NULL` elements of maps, structs and arrays to the `"null"` string while converting maps/struct/array values to strings. The SQL config `spark.sql.legacy.omitNestedNullInCast.enabled` controls the behaviour. When it is `true`, `NULL` elements of structs/maps/arrays will be omitted otherwise, when it is `false`, `NULL` elements will be converted to `"null"`.

### Why are the changes needed?
1. It is impossible to distinguish empty string and null, for instance:
```scala
scala> Seq(Seq(""), Seq(null)).toDF().show
+-----+
|value|
+-----+
|   []|
|   []|
+-----+
```
2. Inconsistent NULL conversions for top-level values and nested columns, for instance:
```scala
scala> sql("select named_struct('c', null), null").show
+---------------------+----+
|named_struct(c, NULL)|NULL|
+---------------------+----+
|                   []|null|
+---------------------+----+
```
3. `.show()` is different from conversions to Hive strings, and as a consequence its output is different from `spark-sql` (sql tests):
```sql
spark-sql> select named_struct('c', null) as struct;
{"c":null}
```
```scala
scala> sql("select named_struct('c', null) as struct").show
+------+
|struct|
+------+
|    []|
+------+
```

4. It is impossible to distinguish empty struct/array from struct/array with null in the current implementation:
```scala
scala> Seq[Seq[String]](Seq(), Seq(null)).toDF.show()
+-----+
|value|
+-----+
|   []|
|   []|
+-----+
```

### Does this PR introduce _any_ user-facing change?
Yes, before:
```scala
scala> Seq(Seq(""), Seq(null)).toDF().show
+-----+
|value|
+-----+
|   []|
|   []|
+-----+
```

After:
```scala
scala> Seq(Seq(""), Seq(null)).toDF().show
+------+
| value|
+------+
|    []|
|[null]|
+------+
```

### How was this patch tested?
By existing test suite `CastSuite`.

Closes #29311 from MaxGekk/nested-null-to-string.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-05 12:03:36 +00:00
Max Gekk 7eb6f45688 [SPARK-32499][SQL] Use {} in conversions maps and structs to strings
### What changes were proposed in this pull request?
Change casting of map and struct values to strings by using the `{}` brackets instead of `[]`. The behavior is controlled by the SQL config `spark.sql.legacy.castComplexTypesToString.enabled`. When it is `true`, `CAST` wraps maps and structs by `[]` in casting to strings. Otherwise, if this is `false`, which is the default, maps and structs are wrapped by `{}`.

### Why are the changes needed?
- To distinguish structs/maps from arrays.
- To make `show`'s output consistent with Hive and conversions to Hive strings.
- To display dataframe content in the same form by `spark-sql` and `show`
- To be consistent with the `*.sql` tests

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

### How was this patch tested?
By existing test suite `CastSuite`.

Closes #29308 from MaxGekk/show-struct-map.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 14:57:09 +00:00
fqaiser94@gmail.com 6d69068057 [SPARK-32521][SQL] Bug-fix: WithFields Expression should not be foldable
### What changes were proposed in this pull request?

Make WithFields Expression not foldable.

### Why are the changes needed?

The following query currently fails on master brach:
```
sql("SELECT named_struct('a', 1, 'b', 2) a")
.select($"a".withField("c", lit(3)).as("a"))
.show(false)
// java.lang.UnsupportedOperationException: Cannot evaluate expression: with_fields(named_struct(a, 1, b, 2), c, 3)
```
This happens because the Catalyst optimizer sees that the WithFields Expression is foldable and tries to statically evaluate the WithFields Expression (via the ConstantFolding rule), however it cannot do so because WithFields Expression is Unevaluable.

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

Yes, queries like the one shared above will now succeed.
That said, this bug was introduced in Spark 3.1.0 which has yet to be released.

### How was this patch tested?

A new unit test was added.

Closes #29338 from fqaiser94/SPARK-32521.

Lead-authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Co-authored-by: fqaiser94 <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 12:11:04 +00:00
gengjiaan 1597d8fcd4 [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
### What changes were proposed in this pull request?
This PR is related to https://github.com/apache/spark/pull/26656.
https://github.com/apache/spark/pull/26656 only support use FILTER clause on aggregate expression without DISTINCT.
This PR will enhance this feature when one or more DISTINCT aggregate expressions which allows the use of the FILTER clause.
Such as:
```
select sum(distinct id) filter (where sex = 'man') from student;
select class_id, sum(distinct id) filter (where sex = 'man') from student group by class_id;
select count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student;
select class_id, count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student group by class_id;
select sum(distinct id), sum(distinct id) filter (where sex = 'man') from student;
select class_id, sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
select class_id, count(id), count(id) filter (where class_id = 1), sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
```

### Why are the changes needed?
Spark SQL only support use FILTER clause on aggregate expression without DISTINCT.
This PR support Filter expression allows simultaneous use of DISTINCT

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

### How was this patch tested?
Exists and new UT

Closes #29291 from beliefer/support-distinct-with-filter.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 04:41:19 +00:00
Max Gekk f3b10f526b [SPARK-32290][SQL][FOLLOWUP] Add version for the SQL config spark.sql.optimizeNullAwareAntiJoin
### What changes were proposed in this pull request?
Add the version `3.1.0` for the SQL config `spark.sql.optimizeNullAwareAntiJoin`.

### Why are the changes needed?
To inform users when the config was added, for example on the page http://spark.apache.org/docs/latest/configuration.html.

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

### How was this patch tested?
By compiling and running `./dev/scalastyle`.

Closes #29335 from MaxGekk/leanken-SPARK-32290-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 16:05:54 +00:00
Takeshi Yamamuro c6109ba918 [SPARK-32257][SQL] Reports explicit errors for invalid usage of SET/RESET command
### What changes were proposed in this pull request?

This PR modified the parser code to handle invalid usages of a SET/RESET command.
For example;
```
SET spark.sql.ansi.enabled true
```
The above SQL command does not change the configuration value and it just tries to display the value of the configuration
`spark.sql.ansi.enabled true`. This PR disallows using special characters including spaces in the configuration name and reports a user-friendly error instead. In the error message, it tells users a workaround to use quotes or a string literal if they still needs to specify a configuration with them. 

Before this PR:
```
scala> sql("SET spark.sql.ansi.enabled true").show(1, -1)
+---------------------------+-----------+
|key                        |value      |
+---------------------------+-----------+
|spark.sql.ansi.enabled true|<undefined>|
+---------------------------+-----------+
```

After this PR:
```
scala> sql("SET spark.sql.ansi.enabled true")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
SET spark.sql.ansi.enabled true
^^^
```

### Why are the changes needed?

For better user-friendly errors.

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

No.

### How was this patch tested?

Added tests in `SparkSqlParserSuite`.

Closes #29146 from maropu/SPARK-32257.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 13:00:07 +00:00
Kent Yao 3deb59d5c2 [SPARK-31709][SQL] Proper base path for database/table location when it is a relative path
### What changes were proposed in this pull request?

Currently, the user home directory is used as the base path for the database and table locations when their locationa are specified with a relative paths, e.g.
```sql
> set spark.sql.warehouse.dir;
spark.sql.warehouse.dir	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/spark-warehouse/
spark-sql> create database loctest location 'loctestdbdir';

spark-sql> desc database loctest;
Database Name	loctest
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir
Owner	kentyao

spark-sql> create table loctest(id int) location 'loctestdbdir';
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	default
Table	loctest
Owner	kentyao
Created Time	Thu May 14 16:29:05 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
```
The user home is not always warehouse-related, unchangeable in runtime, and shared both by database and table as the parent directory. Meanwhile, we use the table path as the parent directory for relative partition locations.

The config `spark.sql.warehouse.dir` represents `the default location for managed databases and tables`.
For databases, the case above seems not to follow its semantics, because it should use ` `spark.sql.warehouse.dir` as the base path instead.

For tables, it seems to be right but here I suggest enriching the meaning that lets it also be the for external tables with relative paths for locations.

With changes in this PR,

The location of a database will be `warehouseDir/dbpath` when `dbpath` is relative.
The location of a table will be `dbpath/tblpath` when `tblpath` is relative.

### Why are the changes needed?

bugfix and improvement

Firstly, the databases with relative locations should be created under the default location specified by `spark.sql.warehouse.dir`.

Secondly, the external tables with relative paths may also follow this behavior for consistency.

At last, the behavior for database, tables and partitions with relative paths to choose base paths should be the same.

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

Yes, this PR changes the `createDatabase`, `alterDatabase`, `createTable` and `alterTable` APIs and related DDLs. If the LOCATION clause is followed by a relative path, the root path will be `spark.sql.warehouse.dir` for databases, and `spark.sql.warehouse.dir` / `dbPath` for tables.

e.g.

#### after
```sql
spark-sql> desc database loctest;
Database Name	loctest
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest
Owner	kentyao
spark-sql> use loctest;
spark-sql> create table loctest(id int) location 'loctest';
20/05/14 18:18:02 WARN InMemoryFileIndex: The directory file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/loctest was not found. Was it deleted very recently?
20/05/14 18:18:02 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, since hive.security.authorization.manager is set to instance of HiveAuthorizerFactory.
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.internal.ss.authz.settings.applied.marker does not exist
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	loctest
Table	loctest
Owner	kentyao
Created Time	Thu May 14 18:18:03 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
spark-sql> alter table loctest set location 'loctest2'
         > ;
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	loctest
Table	loctest
Owner	kentyao
Created Time	Thu May 14 18:18:03 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest2
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
```
### How was this patch tested?

Add unit tests.

Closes #28527 from yaooqinn/SPARK-31709.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 12:48:22 +00:00
beliefer 42f9ee4c7d [SPARK-24884][SQL] Support regexp function regexp_extract_all
### What changes were proposed in this pull request?
`regexp_extract_all` is a very useful function expanded the capabilities of `regexp_extract`.
There are some description of this function.
```
SELECT regexp_extract('1a 2b 14m', '\d+', 0); -- 1
SELECT regexp_extract_all('1a 2b 14m', '\d+', 0); -- [1, 2, 14]
SELECT regexp_extract('1a 2b 14m', '(\d+)([a-z]+)', 2); -- 'a'
SELECT regexp_extract_all('1a 2b 14m', '(\d+)([a-z]+)', 2); -- ['a', 'b', 'm']
```
There are some mainstream database support the syntax.
**Presto:**
https://prestodb.io/docs/current/functions/regexp.html

**Pig:**
https://pig.apache.org/docs/latest/api/org/apache/pig/builtin/REGEX_EXTRACT_ALL.html

Note: This PR pick up the work of https://github.com/apache/spark/pull/21985
### Why are the changes needed?
`regexp_extract_all` is a very useful function and make work easier.

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

### How was this patch tested?
New UT

Closes #27507 from beliefer/support-regexp_extract_all.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 06:03:55 +00:00
Robert (Bobby) Evans 713124d5e3 [SPARK-32274][SQL] Make SQL cache serialization pluggable
### What changes were proposed in this pull request?

Add a config to let users change how SQL/Dataframe data is compressed when cached.

This adds a few new classes/APIs for use with this config.

1. `CachedBatch` is a trait used to tag data that is intended to be cached. It has a few APIs that lets us keep the compression/serialization of the data separate from the metrics about it.
2. `CachedBatchSerializer` provides the APIs that must be implemented to cache data.
    * `convertForCache` is an API that runs a cached spark plan and turns its result into an `RDD[CachedBatch]`.  The actual caching is done outside of this API
   * `buildFilter` is an API that takes a set of predicates and builds a filter function that can be used to filter the `RDD[CachedBatch]` returned by `convertForCache`
   * `decompressColumnar` decompresses an `RDD[CachedBatch]` into an `RDD[ColumnarBatch]` This is only used for a limited set of data types.  These data types may expand in the future.  If they do we can add in a new API with a default value that says which data types this serializer supports.
   * `decompressToRows` decompresses an `RDD[CachedBatch]` into an `RDD[InternalRow]` this API, like `decompressColumnar` decompresses the data in `CachedBatch` but turns it into `InternalRow`s, typically using code generation for performance reasons.

There is also an API that lets you reuse the current filtering based on min/max values. `SimpleMetricsCachedBatch` and `SimpleMetricsCachedBatchSerializer`.

### Why are the changes needed?

This lets users explore different types of compression and compression ratios.

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

This adds in a single config, and exposes some developer API classes described above.

### How was this patch tested?

I ran the unit tests around this and I also did some manual performance tests.  I could find any performance difference between the old and new code, and if there is any it is within error.

Closes #29067 from revans2/pluggable_cache_serializer.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:15:54 +00:00
Yuanjian Li 354313b6bc [SPARK-31894][SS][FOLLOW-UP] Rephrase the config doc
### What changes were proposed in this pull request?
Address comment in https://github.com/apache/spark/pull/28707#discussion_r461102749

### Why are the changes needed?
Hide the implementation details in the config doc.

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

### How was this patch tested?
Document only.

Closes #29315 from xuanyuanking/SPARK-31894-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 14:16:41 +00:00
Takeshi Yamamuro 30e3042dc5 [SPARK-32488][SQL] Use @parser::members and @lexer::members to avoid generating unused code
### What changes were proposed in this pull request?

This PR aims to update `SqlBse.g4` for avoiding generating unused code.
Currently, ANTLR generates unused methods and variables; `isValidDecimal` and `isHint` are only used in the generated lexer. This PR changed the code to use `parser::members` and `lexer::members` to avoid it.

### Why are the changes needed?

To reduce unnecessary code.

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

No.

### How was this patch tested?

Existing tests.

Closes #29296 from maropu/UpdateSqlBase.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 07:51:27 +00:00
Max Gekk 99a855575c [SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources
### What changes were proposed in this pull request?
When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema:
```

### Why are the changes needed?
To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`:
```Scala
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`
```

Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758.

### Does this PR introduce _any_ user-facing change?
Yes. For the example from SPARK-32431:

ORC:
```scala
java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc
	at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329)
	at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3
	at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61)
	at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323)
```

JSON:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

Parquet:
```scala
+------------+
|StructColumn|
+------------+
|     [0,, 1]|
+------------+
```

Avro:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

After the changes, Parquet, ORC, JSON and Avro output the same error:
```scala
Found duplicate column(s) in the data schema: `camelcase`;
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`;
	at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67)
```

### How was this patch tested?
Run modified test suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite"
$ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*"
```
and added new UT to `SchemaUtilsSuite`.

Closes #29234 from MaxGekk/nested-case-insensitive-column.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:05:55 +00:00
Max Gekk d897825d2d [SPARK-32346][SQL] Support filters pushdown in Avro datasource
### What changes were proposed in this pull request?
In the PR, I propose to support pushed down filters in Avro datasource V1 and V2.
1. Added new SQL config `spark.sql.avro.filterPushdown.enabled` to control filters pushdown to Avro datasource. It is on by default.
2. Renamed `CSVFilters` to `OrderedFilters`.
3. `OrderedFilters` is used in `AvroFileFormat` (DSv1) and in `AvroPartitionReaderFactory` (DSv2)
4. Modified `AvroDeserializer` to return None from the `deserialize` method when pushdown filters return `false`.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks up to **2** times on JDK 11:
```
OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                        9614           9669          54          0.1        9614.1       1.0X
pushdown disabled                                 10077          10141          66          0.1       10077.2       1.0X
w/ filters                                         4681           4713          29          0.2        4681.5       2.1X
```

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

### How was this patch tested?
- Added UT to `AvroCatalystDataConversionSuite` and `AvroSuite`
- Re-running `AvroReadBenchmark` using Amazon EC2:

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

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

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
  ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark']
]

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

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

Closes #29145 from MaxGekk/avro-filters-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-30 01:37:42 +08:00
Terry Kim 45b7212fd3 [SPARK-32401][SQL] Migrate function related commands to use UnresolvedFunc to resolve function identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following function related commands to use `UnresolvedFunc` to resolve function identifier:
- DROP FUNCTION
- DESCRIBE FUNCTION
- SHOW FUNCTIONS

`DropFunctionStatement`, `DescribeFunctionStatement` and `ShowFunctionsStatement` logical plans are replaced with `DropFunction`, `DescribeFunction` and `ShowFunctions` logical plans respectively, and each contains `UnresolvedFunc` as its child so that it can be resolved in `Analyzer`.

### Why are the changes needed?

Migrating to the new resolution framework, which resolves `UnresolvedFunc` in `Analyzer`.

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

The message of exception thrown when a catalog is resolved to v2 has been merged to:
`function is only supported in v1 catalog`

Previously, it printed out the command used. E.g.,:
`CREATE FUNCTION is only supported in v1 catalog`

### How was this patch tested?

Updated existing tests.

Closes #29198 from imback82/function_framework.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:55:48 +00:00
Michael Munday a3d80564ad [SPARK-32458][SQL][TESTS] Fix incorrectly sized row value reads
### What changes were proposed in this pull request?
Updates to tests to use correctly sized `getInt` or `getLong` calls.

### Why are the changes needed?
The reads were incorrectly sized (i.e. `putLong` paired with `getInt` and `putInt` paired with `getLong`). This causes test failures on big-endian systems.

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

### How was this patch tested?
Tests were run on a big-endian system (s390x). This change is unlikely to have any practical effect on little-endian systems.

Closes #29258 from mundaym/fix-row.

Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-28 10:36:20 -07:00
yi.wu ca1ecf7f9f [SPARK-32459][SQL] Support WrappedArray as customCollectionCls in MapObjects
### What changes were proposed in this pull request?

This PR supports `WrappedArray` as `customCollectionCls` in `MapObjects`.

### Why are the changes needed?

This helps fix the regression caused by SPARK-31826.  For the following test, it can pass in branch-3.0 but fail in master branch:

```scala
test("WrappedArray") {
    val myUdf = udf((a: WrappedArray[Int]) =>
      WrappedArray.make[Int](Array(a.head + 99)))
    checkAnswer(Seq(Array(1))
      .toDF("col")
      .select(myUdf(Column("col"))),
      Row(ArrayBuffer(100)))
  }
```

In SPARK-31826, we've changed the catalyst-to-scala converter from `CatalystTypeConverters` to `ExpressionEncoder.deserializer`. However, `CatalystTypeConverters` supports `WrappedArray` while `ExpressionEncoder.deserializer` doesn't.

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

No,  SPARK-31826 is merged into master and branch-3.1, which haven't been released.

### How was this patch tested?

Added a new test for `WrappedArray` in `UDFSuite`; Also updated `ObjectExpressionsSuite` for `MapObjects`.

Closes #29261 from Ngone51/fix-wrappedarray.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:24:15 +00:00
xuewei.linxuewei 12b9787a7f [SPARK-32290][SQL] SingleColumn Null Aware Anti Join Optimize
### What changes were proposed in this pull request?
Normally, a Null aware anti join will be planed into BroadcastNestedLoopJoin which is very time consuming, for instance, in TPCH Query 16.

```
select
    p_brand,
    p_type,
    p_size,
    count(distinct ps_suppkey) as supplier_cnt
from
    partsupp,
    part
where
    p_partkey = ps_partkey
    and p_brand <> 'Brand#45'
    and p_type not like 'MEDIUM POLISHED%'
    and p_size in (49, 14, 23, 45, 19, 3, 36, 9)
    and ps_suppkey not in (
        select
            s_suppkey
        from
            supplier
        where
            s_comment like '%Customer%Complaints%'
    )
group by
    p_brand,
    p_type,
    p_size
order by
    supplier_cnt desc,
    p_brand,
    p_type,
    p_size
```

In above query, will planed into

LeftAnti
    condition Or((ps_suppkey=s_suppkey), IsNull(ps_suppkey=s_suppkey))

Inside BroadcastNestedLoopJoinExec will perform O(M\*N), BUT if there is only single column in NAAJ, we can always change buildSide into a HashSet, and streamedSide just need to lookup in the HashSet, then the calculation will be optimized into O(M).

But this optimize is only targeting on null aware anti join with single column case, because multi-column support is much more complicated, we might be able to support multi-column in future.
After apply this patch, the TPCH Query 16 performance decrease from 41mins to 30s

The semantic of null-aware anti join is:

![image](https://user-images.githubusercontent.com/17242071/88077041-66a39a00-cbad-11ea-8fb6-c235c4d219b4.png)

### Why are the changes needed?
TPCH is a common benchmark for distributed compute engine, all other 21 Query works fine on Spark, except for Query 16, apply this patch will make Spark more competitive among all these popular engine. BTW, this patch has restricted rules and only apply on NAAJ Single Column case, which is safe enough.

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

### How was this patch tested?
1. SQLQueryTestSuite with NOT IN keyword SQL, add CONFIG_DIM with spark.sql.optimizeNullAwareAntiJoin on and off
2. added case in org.apache.spark.sql.JoinSuite.
3. added case in org.apache.spark.sql.SubquerySuite.
3. Compare performance before and after applying this patch against TPCH Query 16.
4. config combination against e2e test with following

```
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "sspark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "true"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "true"
)
```

Closes #29104 from leanken/leanken-SPARK-32290.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 04:42:15 +00:00