### What changes were proposed in this pull request?
This PR proposes to add `as` API to RelationalGroupedDataset. It creates KeyValueGroupedDataset instance using given grouping expressions, instead of a typed function in groupByKey API. Because it can leverage existing columns, it can use existing data partition, if any, when doing operations like cogroup.
### Why are the changes needed?
Currently if users want to do cogroup on DataFrames, there is no good way to do except for KeyValueGroupedDataset.
1. KeyValueGroupedDataset ignores existing data partition if any. That is a problem.
2. groupByKey calls typed function to create additional keys. You can not reuse existing columns, if you just need grouping by them.
```scala
// df1 and df2 are certainly partitioned and sorted.
val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c")
.repartition($"a").sortWithinPartitions("a")
val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a", "b", "c")
.repartition($"a").sortWithinPartitions("a")
```
```scala
// This groupBy.as.cogroup won't unnecessarily repartition the data
val df3 = df1.groupBy("a").as[Int]
.cogroup(df2.groupBy("a").as[Int]) { case (key, data1, data2) =>
data1.zip(data2).map { p =>
p._1.getInt(2) + p._2.getInt(2)
}
}
```
```
== Physical Plan ==
*(5) SerializeFromObject [input[0, int, false] AS value#11247]
+- CoGroup org.apache.spark.sql.DataFrameSuite$$Lambda$4922/12067092816eec1b6f, a#11209: int, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [a#11209], [a#11225], [a#11209, b#11210, c#11211], [a#11225, b#11226, c#11227], obj#11246: int
:- *(2) Sort [a#11209 ASC NULLS FIRST], false, 0
: +- Exchange hashpartitioning(a#11209, 5), false, [id=#10218]
: +- *(1) Project [_1#11202 AS a#11209, _2#11203 AS b#11210, _3#11204 AS c#11211]
: +- *(1) LocalTableScan [_1#11202, _2#11203, _3#11204]
+- *(4) Sort [a#11225 ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(a#11225, 5), false, [id=#10223]
+- *(3) Project [_1#11218 AS a#11225, _2#11219 AS b#11226, _3#11220 AS c#11227]
+- *(3) LocalTableScan [_1#11218, _2#11219, _3#11220]
```
```scala
// Current approach creates additional AppendColumns and repartition data again
val df4 = df1.groupByKey(r => r.getInt(0)).cogroup(df2.groupByKey(r => r.getInt(0))) {
case (key, data1, data2) =>
data1.zip(data2).map { p =>
p._1.getInt(2) + p._2.getInt(2)
}
}
```
```
== Physical Plan ==
*(7) SerializeFromObject [input[0, int, false] AS value#11257]
+- CoGroup org.apache.spark.sql.DataFrameSuite$$Lambda$4933/138102700737171997, value#11252: int, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [value#11252], [value#11254], [a#11209, b#11210, c#11211], [a#11225, b#11226, c#11227], obj#11256: int
:- *(3) Sort [value#11252 ASC NULLS FIRST], false, 0
: +- Exchange hashpartitioning(value#11252, 5), true, [id=#10302]
: +- AppendColumns org.apache.spark.sql.DataFrameSuite$$Lambda$4930/19529195347ce07f47, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [input[0, int, false] AS value#11252]
: +- *(2) Sort [a#11209 ASC NULLS FIRST], false, 0
: +- Exchange hashpartitioning(a#11209, 5), false, [id=#10297]
: +- *(1) Project [_1#11202 AS a#11209, _2#11203 AS b#11210, _3#11204 AS c#11211]
: +- *(1) LocalTableScan [_1#11202, _2#11203, _3#11204]
+- *(6) Sort [value#11254 ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(value#11254, 5), true, [id=#10312]
+- AppendColumns org.apache.spark.sql.DataFrameSuite$$Lambda$4932/15265288491f0e0c1f, createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [input[0, int, false] AS value#11254]
+- *(5) Sort [a#11225 ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(a#11225, 5), false, [id=#10307]
+- *(4) Project [_1#11218 AS a#11225, _2#11219 AS b#11226, _3#11220 AS c#11227]
+- *(4) LocalTableScan [_1#11218, _2#11219, _3#11220]
```
### Does this PR introduce any user-facing change?
Yes, this adds a new `as` API to RelationalGroupedDataset. Users can use it to create KeyValueGroupedDataset and do cogroup.
### How was this patch tested?
Unit tests.
Closes#26509 from viirya/SPARK-29427-2.
Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
A few cleanups for https://github.com/apache/spark/pull/26516:
1. move the calculating of partition start indices from the RDD to the rule. We can reuse code from "shrink number of reducers" in the future if we split partitions by size.
2. only check extra shuffles when adding local readers to the probe side.
3. add comments.
4. simplify the config name: `optimizedLocalShuffleReader` -> `localShuffleReader`
### Why are the changes needed?
make code more maintainable.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26625 from cloud-fan/aqe.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
### What changes were proposed in this pull request?
Modify `UTF8String.toInt/toLong` to support trim spaces for both sides before converting it to byte/short/int/long.
With this kind of "cheap" trim can help improve performance for casting string to integrals. The idea is from https://github.com/apache/spark/pull/24872#issuecomment-556917834
### Why are the changes needed?
make the behavior consistent.
### Does this PR introduce any user-facing change?
yes, cast string to an integral type, and binary comparison between string and integrals will trim spaces first. their behavior will be consistent with float and double.
### How was this patch tested?
1. add ut.
2. benchmark tests
the benchmark is modified based on https://github.com/apache/spark/pull/24872#issuecomment-503827016
```scala
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution.benchmark
import org.apache.spark.benchmark.Benchmark
/**
* Benchmark trim the string when casting string type to Boolean/Numeric types.
* To run this benchmark:
* {{{
* 1. without sbt:
* bin/spark-submit --class <this class> --jars <spark core test jar> <spark sql test jar>
* 2. build/sbt "sql/test:runMain <this class>"
* 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
* Results will be written to "benchmarks/CastBenchmark-results.txt".
* }}}
*/
object CastBenchmark extends SqlBasedBenchmark {
This conversation was marked as resolved by yaooqinn
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
val title = "Cast String to Integral"
runBenchmark(title) {
withTempPath { dir =>
val N = 500L << 14
val df = spark.range(N)
val types = Seq("int", "long")
(1 to 5).by(2).foreach { i =>
df.selectExpr(s"concat(id, '${" " * i}') as str")
.write.mode("overwrite").parquet(dir + i.toString)
}
val benchmark = new Benchmark(title, N, minNumIters = 5, output = output)
Seq(true, false).foreach { trim =>
types.foreach { t =>
val str = if (trim) "trim(str)" else "str"
val expr = s"cast($str as $t) as c_$t"
(1 to 5).by(2).foreach { i =>
benchmark.addCase(expr + s" - with $i spaces") { _ =>
spark.read.parquet(dir + i.toString).selectExpr(expr).collect()
}
}
}
}
benchmark.run()
}
}
}
}
```
#### benchmark result.
normal trim v.s. trim in toInt/toLong
```java
================================================================================================
Cast String to Integral
================================================================================================
Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1
Intel(R) Core(TM) i5-5287U CPU 2.90GHz
Cast String to Integral: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
cast(trim(str) as int) as c_int - with 1 spaces 10220 12994 1337 0.8 1247.5 1.0X
cast(trim(str) as int) as c_int - with 3 spaces 4763 8356 357 1.7 581.4 2.1X
cast(trim(str) as int) as c_int - with 5 spaces 4791 8042 NaN 1.7 584.9 2.1X
cast(trim(str) as long) as c_long - with 1 spaces 4014 6755 NaN 2.0 490.0 2.5X
cast(trim(str) as long) as c_long - with 3 spaces 4737 6938 NaN 1.7 578.2 2.2X
cast(trim(str) as long) as c_long - with 5 spaces 4478 6919 1404 1.8 546.6 2.3X
cast(str as int) as c_int - with 1 spaces 4443 6222 NaN 1.8 542.3 2.3X
cast(str as int) as c_int - with 3 spaces 3659 3842 170 2.2 446.7 2.8X
cast(str as int) as c_int - with 5 spaces 4372 7996 NaN 1.9 533.7 2.3X
cast(str as long) as c_long - with 1 spaces 3866 5838 NaN 2.1 471.9 2.6X
cast(str as long) as c_long - with 3 spaces 3793 5449 NaN 2.2 463.0 2.7X
cast(str as long) as c_long - with 5 spaces 4947 5961 1198 1.7 603.9 2.1X
```
Closes#26622 from yaooqinn/cheapstringtrim.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
allow the sql test files to specify different dimensions of config sets during testing. For example,
```
--CONFIG_DIM1 a=1
--CONFIG_DIM1 b=2,c=3
--CONFIG_DIM2 x=1
--CONFIG_DIM2 y=1,z=2
```
This example defines 2 config dimensions, and each dimension defines 2 config sets. We will run the queries 4 times:
1. a=1, x=1
2. a=1, y=1, z=2
3. b=2, c=3, x=1
4. b=2, c=3, y=1, z=2
### Why are the changes needed?
Currently `SQLQueryTestSuite` takes a long time. This is because we run each test at least 3 times, to check with different codegen modes. This is not necessary for most of the tests, e.g. DESC TABLE. We should only check these codegen modes for certain tests.
With the --CONFIG_DIM directive, we can do things like: test different join operator(broadcast or shuffle join) X different codegen modes.
After reducing testing time, we should be able to run thrifter server SQL tests with config settings.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
test only
Closes#26612 from cloud-fan/test.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Make `ResolveRelations` call `ResolveTables` at the beginning, and make `ResolveTables` call `ResolveTempViews`(newly added) at the beginning, to ensure the relation resolution priority.
### Why are the changes needed?
To resolve an `UnresolvedRelation`, the general process is:
1. try to resolve to (global) temp view first. If it's not a temp view, move on
2. if the table name specifies a catalog, lookup the table from the specified catalog. Otherwise, lookup table from the current catalog.
3. when looking up table from session catalog, return a v1 relation if the table provider is v1.
Currently, this process is done by 2 rules: `ResolveTables` and `ResolveRelations`. To avoid rule conflicts, we add a lot of checks:
1. `ResolveTables` only resolves `UnresolvedRelation` if it's not a temp view and the resolved table is not v1.
2. `ResolveRelations` only resolves `UnresolvedRelation` if the table name has less than 2 parts.
This requires to run `ResolveTables` before `ResolveRelations`, otherwise we may resolve a v2 table to a v1 relation.
To clearly guarantee the resolution priority, and avoid massive changes, this PR proposes to call one rule in another rule to ensure the rule execution order. Now the process is simple:
1. first run `ResolveTempViews`, see if we can resolve relation to temp view
2. then run `ResolveTables`, see if we can resolve relation to v2 tables.
3. finally run `ResolveRelations`, see if we can resolve relation to v1 tables.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26214 from cloud-fan/resolve.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Ryan Blue <blue@apache.org>
### What changes were proposed in this pull request?
When implementing a ScanBuilder, we require the implementor to provide the schema of the data and the number of partitions.
However, when someone is implementing WriteBuilder we only pass them the schema, but not the number of partitions. This is an asymetrical developer experience.
This PR adds a PhysicalWriteInfo interface that is passed to createBatchWriterFactory and createStreamingWriterFactory that adds the number of partitions of the data that is going to be written.
### Why are the changes needed?
Passing in the number of partitions on the WriteBuilder would enable data sources to provision their write targets before starting to write. For example:
it could be used to provision a Kafka topic with a specific number of partitions
it could be used to scale a microservice prior to sending the data to it
it could be used to create a DsV2 that sends the data to another spark cluster (currently not possible since the reader wouldn't be able to know the number of partitions)
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Tests passed
Closes#26591 from edrevo/temp.
Authored-by: Ximo Guanter <joaquin.guantergonzalbez@telefonica.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is to refactor `SparkPlan` code; it mainly removed `newMutableProjection`/`newOrdering`/`newNaturalAscendingOrdering` from `SparkPlan`.
The other modifications are listed below;
- Move `BaseOrdering` from `o.a.s.sqlcatalyst.expressions.codegen.GenerateOrdering.scala` to `o.a.s.sqlcatalyst.expressions.ordering.scala`
- `RowOrdering` extends `CodeGeneratorWithInterpretedFallback ` for `BaseOrdering`
- Remove the unused variables (`subexpressionEliminationEnabled` and `codeGenFallBack`) from `SparkPlan`
### Why are the changes needed?
For better code/test coverage.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing.
Closes#26615 from maropu/RefactorOrdering.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In origin way to judge if a DataSet is empty by
```
def isEmpty: Boolean = withAction("isEmpty", limit(1).groupBy().count().queryExecution) { plan =>
plan.executeCollect().head.getLong(0) == 0
}
```
will add two shuffles by `limit()`, `groupby() and count()`, then collect all data to driver.
In this way we can avoid `oom` when collect data to driver. But it will trigger all partitions calculated and add more shuffle process.
We change it to
```
def isEmpty: Boolean = withAction("isEmpty", select().queryExecution) { plan =>
plan.executeTake(1).isEmpty
}
```
After these pr, we will add a column pruning to origin LogicalPlan and use `executeTake()` API.
then we won't add more shuffle process and just compute only one partition's data in last stage.
In this way we can reduce cost when we call `DataSet.isEmpty()` and won't bring memory issue to driver side.
### Why are the changes needed?
Optimize Dataset.isEmpty()
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Origin UT
Closes#26500 from AngersZhuuuu/SPARK-29874.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add typeof function for Spark to get the underlying type of value.
```sql
-- !query 0
select typeof(1)
-- !query 0 schema
struct<typeof(1):string>
-- !query 0 output
int
-- !query 1
select typeof(1.2)
-- !query 1 schema
struct<typeof(1.2):string>
-- !query 1 output
decimal(2,1)
-- !query 2
select typeof(array(1, 2))
-- !query 2 schema
struct<typeof(array(1, 2)):string>
-- !query 2 output
array<int>
-- !query 3
select typeof(a) from (values (1), (2), (3.1)) t(a)
-- !query 3 schema
struct<typeof(a):string>
-- !query 3 output
decimal(11,1)
decimal(11,1)
decimal(11,1)
```
##### presto
```sql
presto> select typeof(array[1]);
_col0
----------------
array(integer)
(1 row)
```
##### PostgreSQL
```sql
postgres=# select pg_typeof(a) from (values (1), (2), (3.0)) t(a);
pg_typeof
-----------
numeric
numeric
numeric
(3 rows)
```
##### impala
https://issues.apache.org/jira/browse/IMPALA-1597
### Why are the changes needed?
a function which is better we have to help us debug, test, develop ...
### Does this PR introduce any user-facing change?
add a new function
### How was this patch tested?
add ut and example
Closes#26599 from yaooqinn/SPARK-29961.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
I propose to add a test from the commit a936522113 for 2.4. I extended the test by a few more lengths of requested field to cover more code branches in Jackson Core. In particular, [the optimization](5eb8973f87/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala (L473-L476)) calls Jackson's method 42b8b56684/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java (L742-L746) where the internal buffer size is **8000**. In this way:
- 2000 to check 2000+2000+2000 < 8000
- 2800 from the 2.4 commit. It covers the specific case: 42b8b56684/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java (L746)
- 8000-1, 8000, 8000+1 are sizes around the size of the internal buffer
- 65535 to test an outstanding large field.
### Why are the changes needed?
To be sure that the current implementation and future versions of Spark don't have the bug fixed in 2.4.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By running `JsonFunctionsSuite`.
Closes#26613 from MaxGekk/json_tuple-test.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Use JUnit assertions in tests uniformly, not JVM assert() statements.
### Why are the changes needed?
assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.
### Does this PR introduce any user-facing change?
No. The assertion logic should be identical.
### How was this patch tested?
Existing tests.
Closes#26581 from srowen/assertToJUnit.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
Fix the inconsistent behavior of build-in function SQL LEFT/RIGHT.
### Why are the changes needed?
As the comment in https://github.com/apache/spark/pull/26497#discussion_r345708065, Postgre dialect should not be affected by the ANSI mode config.
During reran the existing tests, only the LEFT/RIGHT build-in SQL function broke the assumption. We fix this by following https://www.postgresql.org/docs/12/sql-keywords-appendix.html: `LEFT/RIGHT reserved (can be function or type)`
### Does this PR introduce any user-facing change?
Yes, the Postgre dialect will not be affected by the ANSI mode config.
### How was this patch tested?
Existing UT.
Closes#26584 from xuanyuanking/SPARK-29951.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
SPARK-28885(#26107) has supported the ANSI store assignment rules and stopped running some ported PgSQL regression tests that violate the rules. To re-activate these tests, this pr is to modify them for passing tests with the rules.
### Why are the changes needed?
To make the test coverage better.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#26492 from maropu/SPARK-28885-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
The Web UI SQL Tab provides information on the executed SQL using plan graphs and by reporting SQL execution plans. Both sources provide useful information. Physical execution plans report Codegen Stage Ids. This PR adds Codegen Stage Ids to the plan graphs.
### Why are the changes needed?
It is useful to have Codegen Stage Id information also reported in plan graphs, this allows to more easily match physical plans and graphs with metrics when troubleshooting SQL execution.
Example snippet to show the proposed change:
![](https://issues.apache.org/jira/secure/attachment/12985837/snippet__plan_graph_with_Codegen_Stage_Id_Annotated.png)
Example of the current state:
![](https://issues.apache.org/jira/secure/attachment/12985838/snippet_plan_graph_before_patch.png)
Physical plan:
![](https://issues.apache.org/jira/secure/attachment/12985932/Physical_plan_Annotated.png)
### Does this PR introduce any user-facing change?
This PR adds Codegen Stage Id information to SQL plan graphs in the Web UI/SQL Tab.
### How was this patch tested?
Added a test + manually tested
Closes#26519 from LucaCanali/addCodegenStageIdtoWEBUIGraphs.
Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is to refactor Predicate code; it mainly removed `newPredicate` from `SparkPlan`.
Modifications are listed below;
- Move `Predicate` from `o.a.s.sqlcatalyst.expressions.codegen.GeneratePredicate.scala` to `o.a.s.sqlcatalyst.expressions.predicates.scala`
- To resolve the name conflict, rename `o.a.s.sqlcatalyst.expressions.codegen.Predicate` to `o.a.s.sqlcatalyst.expressions.BasePredicate`
- Extend `CodeGeneratorWithInterpretedFallback ` for `BasePredicate`
This comes from the cloud-fan suggestion: https://github.com/apache/spark/pull/26420#discussion_r348005497
### Why are the changes needed?
For better code/test coverage.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#26604 from maropu/RefactorPredicate.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Remove the special handling of the negative sign in the parser (interval literal and type constructor)
### Why are the changes needed?
The negative sign is an operator (UnaryMinus). We don't need to handle it specially, which is kind of doing constant folding at parser side.
### Does this PR introduce any user-facing change?
The error message becomes a little different. Now it reports type mismatch for the `-` operator.
### How was this patch tested?
existing tests
Closes#26578 from cloud-fan/interval.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
In the PR, I propose to add tests from the commit 47cb1f359a for Spark 2.4 that check formatting of timestamp strings for various seconds fractions.
### Why are the changes needed?
To make sure that current behavior is the same as in Spark 2.4
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`.
Closes#26601 from MaxGekk/format-timestamp-micros-tests.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
`AdaptiveSparkPlanExec` should forward `executeCollect` and `executeTake` to the underlying physical plan.
### Why are the changes needed?
some physical plan has optimization in `executeCollect` and `executeTake`. For example, `CollectLimitExec` won't do shuffle for outermost limit.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
a new test
This closes#26560Closes#26576 from cloud-fan/aqe.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
### What changes were proposed in this pull request?
Exception improvement.
### Why are the changes needed?
After selecting pgSQL dialect, queries which are failing because of wrong syntax will give long exception stack trace. For example,
`explain select cast ("abc" as boolean);`
Current output:
> ERROR SparkSQLDriver: Failed in [explain select cast ("abc" as boolean)]
> java.lang.IllegalArgumentException: invalid input syntax for type boolean: abc
> at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$2(PostgreCastToBoolean.scala:51)
> at org.apache.spark.sql.catalyst.expressions.CastBase.buildCast(Cast.scala:277)
> at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$1(PostgreCastToBoolean.scala:44)
> at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeEval(Cast.scala:773)
> at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:460)
> at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
> at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
> at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:286)
> at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:286)
> at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:291)
> at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
> at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
> at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
> at org.apache.spark.sql.catalyst.plans.QueryPlan.
> .
> .
> .
### Does this PR introduce any user-facing change?
Yes. After this PR, output for above query will be:
> == Physical Plan ==
> org.apache.spark.sql.AnalysisException: invalid input syntax for type boolean: abc;
>
> Time taken: 0.044 seconds, Fetched 1 row(s)
> 19/11/15 15:38:57 INFO SparkSQLCLIDriver: Time taken: 0.044 seconds, Fetched 1 row(s)
### How was this patch tested?
Updated existing test cases.
Closes#26546 from jobitmathew/pgsqlexception.
Authored-by: Jobit Mathew <jobit.mathew@huawei.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR update the local reader task number from 1 to multi `partitionStartIndices.length`.
### Why are the changes needed?
Improve the performance of local shuffle reader.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Existing UTs
Closes#26516 from JkSelf/improveLocalShuffleReader.
Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR try to make sure the comparison results of `compared by 8 bytes at a time` and `compared by bytes wise` in RecordBinaryComparator is *consistent*, by reverse long bytes if it is little-endian and using Long.compareUnsigned.
### Why are the changes needed?
If the architecture supports unaligned or the offset is 8 bytes aligned, `RecordBinaryComparator` compare 8 bytes at a time by reading 8 bytes as a long. Related code is
```
if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) {
while (i <= leftLen - 8) {
final long v1 = Platform.getLong(leftObj, leftOff + i);
final long v2 = Platform.getLong(rightObj, rightOff + i);
if (v1 != v2) {
return v1 > v2 ? 1 : -1;
}
i += 8;
}
}
```
Otherwise, it will compare bytes by bytes. Related code is
```
while (i < leftLen) {
final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff;
final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff;
if (v1 != v2) {
return v1 > v2 ? 1 : -1;
}
i += 1;
}
```
However, on little-endian machine, the result of *compared by a long value* and *compared bytes by bytes* maybe different.
For two same records, its offsets may vary in the first run and second run, which will lead to compare them using long comparison or byte-by-byte comparison, the result maybe different.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Add new test cases in RecordBinaryComparatorSuite
Closes#26548 from WangGuangxin/binary_comparator.
Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Update `Literal.sql` to make date, timestamp and interval consistent. They should all use the `TYPE 'value'` format.
### Why are the changes needed?
Make the default alias consistent. For example, without this patch we will see
```
scala> sql("select interval '1 day', date '2000-10-10'").show
+------+-----------------+
|1 days|DATE '2000-10-10'|
+------+-----------------+
|1 days| 2000-10-10|
+------+-----------------+
```
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26579 from cloud-fan/sql.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR adds `ALTER TABLE a.b.c RENAME TO x.y.x` support for V2 catalogs.
### Why are the changes needed?
The current implementation doesn't support this command V2 catalogs.
### Does this PR introduce any user-facing change?
Yes, now the renaming table works for v2 catalogs:
```
scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show
+---------+---------+
|namespace|tableName|
+---------+---------+
| ns1.ns2| old|
+---------+---------+
scala> spark.sql("ALTER TABLE testcat.ns1.ns2.old RENAME TO testcat.ns1.ns2.new").show
scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show
+---------+---------+
|namespace|tableName|
+---------+---------+
| ns1.ns2| new|
+---------+---------+
```
### How was this patch tested?
Added unit tests.
Closes#26539 from imback82/rename_table.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR is a followup of https://github.com/apache/spark/pull/26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency.
### Why are the changes needed?
To put the similar configurations together and for readability.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Manually tested as described in https://github.com/apache/spark/pull/26530.
Closes#26570 from HyukjinKwon/SPARK-25694.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
When regenerating golden files, the set operations via `--SET` will not be done, but those with --import should be exceptions because we need the set command.
### Why are the changes needed?
fix test tool.
### Does this PR introduce any user-facing change?
### How was this patch tested?
add ut, but I'm not sure we need these tests for tests itself.
cc maropu cloud-fan
Closes#26557 from yaooqinn/SPARK-29873.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add AlterNamespaceSetLocationStatement, AlterNamespaceSetLocation, AlterNamespaceSetLocationExec to make ALTER DATABASE (SET LOCATION) look up catalog like v2 commands.
And also refine the code of AlterNamespaceSetProperties, AlterNamespaceSetPropertiesExec, DescribeNamespace, DescribeNamespaceExec to use SupportsNamespaces instead of CatalogPlugin for catalog parameter.
### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.
### Does this PR introduce any user-facing change?
Yes, add "ALTER NAMESPACE ... SET LOCATION" whose function is same as "ALTER DATABASE ... SET LOCATION" and "ALTER SCHEMA ... SET LOCATION".
### How was this patch tested?
New unit tests
Closes#26562 from fuwhu/SPARK-29859.
Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
We now have two different implementation for multi-units interval strings to CalendarInterval type values.
One is used to covert interval string literals to CalendarInterval. This approach will re-delegate the interval string to spark parser which handles the string as a `singleInterval` -> `multiUnitsInterval` -> eventually call `IntervalUtils.fromUnitStrings`
The other is used in `Cast`, which eventually calls `IntervalUtils.stringToInterval`. This approach is ~10 times faster than the other.
We should unify these two for better performance and simple logic. this pr uses the 2nd approach.
### Why are the changes needed?
We should unify these two for better performance and simple logic.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
we shall not fail on existing uts
Closes#26491 from yaooqinn/SPARK-29870.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table.
Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval
-- | -- | -- | -- | --
Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds
SQL STANDARD | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06
ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S
### Why are the changes needed?
for ANSI SQL support
### Does this PR introduce any user-facing change?
yes,interval out now has 3 output styles
### How was this patch tested?
add new unit tests
cc cloud-fan maropu MaxGekk HyukjinKwon thanks.
Closes#26418 from yaooqinn/SPARK-29783.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
I've noticed that there are two functions to sort arrays sort_array and array_sort.
sort_array is from 1.5.0 and it has the possibility of ordering both ascending and descending
array_sort is from 2.4.0 and it only has the possibility of ordering in ascending.
Basically I just added the possibility of ordering either ascending or descending using array_sort.
I think it would be good to have unified behaviours and not having to user sort_array when you want to order in descending order.
Imagine that you are new to spark, I'd like to be able to sort array using the newest spark functions.
### Why are the changes needed?
Basically to be able to sort the array in descending order using *array_sort* instead of using *sort_array* from 1.5.0
### Does this PR introduce any user-facing change?
Yes, now you are able to sort the array in descending order. Note that it has the same behaviour with nulls than sort_array
### How was this patch tested?
Test's added
This is the link to the [jira](https://issues.apache.org/jira/browse/SPARK-29020)
Closes#25728 from Gschiavon/improving-array-sort.
Lead-authored-by: gschiavon <german.schiavon@lifullconnect.com>
Co-authored-by: Takuya UESHIN <ueshin@databricks.com>
Co-authored-by: gschiavon <Gschiavon@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory`
### Why are the changes needed?
This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](https://github.com/apache/spark/pull/17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users.
### Does this PR introduce any user-facing change?
Yes. This provides a new user-configurable property.
By default, the behavior is unchanged.
### How was this patch tested?
Manual testing.
**BEFORE**
```
$ build/sbt package
$ bin/spark-shell
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+
scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
java.lang.Error: factory already defined
at java.net.URL.setURLStreamHandlerFactory(URL.java:1134)
... 47 elided
```
**AFTER**
```
$ build/sbt package
$ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+
scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
```
Closes#26530 from jiangzho/master.
Lead-authored-by: Zhou Jiang <zhou_jiang@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: zhou-jiang <zhou_jiang@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
### What changes were proposed in this pull request?
SPARK-27444 introduced `dmlStatementNoWith` so that any dml that needs cte support can leverage it. It be better if we move DELETE/UPDATE/MERGE rules to `dmlStatementNoWith`.
### Why are the changes needed?
Wit this change, we can support syntax like "With t AS (SELECT) DELETE FROM xxx", and so as UPDATE/MERGE.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
New cases added.
Closes#26536 from xianyinxin/SPARK-29907.
Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add AlterNamespaceSetPropertiesStatement, AlterNamespaceSetProperties and AlterNamespaceSetPropertiesExec to make ALTER DATABASE (SET DBPROPERTIES) command look up catalog like v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.
### Does this PR introduce any user-facing change?
Yes, add "ALTER NAMESPACE ... SET (DBPROPERTIES | PROPERTIES) ..." whose function is same as "ALTER DATABASE ... SET DBPROPERTIES ..." and "ALTER SCHEMA ... SET DBPROPERTIES ...".
### How was this patch tested?
New unit test
Closes#26551 from fuwhu/SPARK-29858.
Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
In the PR, I propose to add tests from the commit 9c7e8be1dc for Spark 2.4 that check parsing of timestamp strings for various seconds fractions.
### Why are the changes needed?
To make sure that current behavior is the same as in Spark 2.4
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`.
Closes#26558 from MaxGekk/parse-timestamp-micros-tests.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"
### Why are the changes needed?
The relation between "spark.sql.ansi.enabled" and "spark.sql.dialect" is confusing, since the "PostgreSQL" dialect should contain the features of "spark.sql.ansi.enabled".
To make things clearer, we can rename the "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled", thus the option "spark.sql.dialect.spark.ansi.enabled" is only for Spark dialect.
For the casting and arithmetic operations, runtime exceptions should be thrown if "spark.sql.dialect" is "spark" and "spark.sql.dialect.spark.ansi.enabled" is true or "spark.sql.dialect" is PostgresSQL.
### Does this PR introduce any user-facing change?
Yes, the config name changed.
### How was this patch tested?
Existing UT.
Closes#26444 from xuanyuanking/SPARK-29807.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to add `io.netty.tryReflectionSetAccessible=true` to the testing configuration for JDK11 because this is an officially documented requirement of Apache Arrow.
Apache Arrow community documented this requirement at `0.15.0` ([ARROW-6206](https://github.com/apache/arrow/pull/5078)).
> #### For java 9 or later, should set "-Dio.netty.tryReflectionSetAccessible=true".
> This fixes `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available`. thrown by netty.
### Why are the changes needed?
After ARROW-3191, Arrow Java library requires the property `io.netty.tryReflectionSetAccessible` to be set to true for JDK >= 9. After https://github.com/apache/spark/pull/26133, JDK11 Jenkins job seem to fail.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/676/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/677/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/678/
```scala
Previous exception in task:
sun.misc.Unsafe or java.nio.DirectByteBuffer.<init>(long, int) not available

io.netty.util.internal.PlatformDependent.directBuffer(PlatformDependent.java:473)

io.netty.buffer.NettyArrowBuf.getDirectBuffer(NettyArrowBuf.java:243)

io.netty.buffer.NettyArrowBuf.nioBuffer(NettyArrowBuf.java:233)

io.netty.buffer.ArrowBuf.nioBuffer(ArrowBuf.java:245)

org.apache.arrow.vector.ipc.message.ArrowRecordBatch.computeBodyLength(ArrowRecordBatch.java:222)

```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Pass the Jenkins with JDK11.
Closes#26552 from dongjoon-hyun/SPARK-ARROW-JDK11.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This pr is to remove floating-point `Sum/Average/CentralMomentAgg` from order-insensitive aggregates in `EliminateSorts`.
This pr comes from the gatorsmile suggestion: https://github.com/apache/spark/pull/26011#discussion_r344583899
### Why are the changes needed?
Bug fix.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Added tests in `SubquerySuite`.
Closes#26534 from maropu/SPARK-29343-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Add DescribeNamespaceStatement, DescribeNamespace and DescribeNamespaceExec
to make "DESC DATABASE" look up catalog like v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.
### Does this PR introduce any user-facing change?
Yes, add "DESC NAMESPACE" whose function is same as "DESC DATABASE" and "DESC SCHEMA".
### How was this patch tested?
New unit test
Closes#26513 from fuwhu/SPARK-29834.
Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR proposes to show Python, pandas and PyArrow versions in integrated UDF tests as a clue so when the test cases fail, it show the related version information.
I think we don't really need this kind of version information in the test case name for now since I intend that integrated SQL test cases do not target to test different combinations of Python, Pandas and PyArrow.
### Why are the changes needed?
To make debug easier.
### Does this PR introduce any user-facing change?
It will change test name to include related Python, pandas and PyArrow versions.
### How was this patch tested?
Manually tested:
```
[info] - udf/postgreSQL/udf-case.sql - Scala UDF *** FAILED *** (8 seconds, 229 milliseconds)
[info] udf/postgreSQL/udf-case.sql - Scala UDF
...
[info] - udf/postgreSQL/udf-case.sql - Regular Python UDF *** FAILED *** (6 seconds, 298 milliseconds)
[info] udf/postgreSQL/udf-case.sql - Regular Python UDF
[info] Python: 3.7
...
[info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF *** FAILED *** (6 seconds, 376 milliseconds)
[info] udf/postgreSQL/udf-case.sql - Scalar Pandas UDF
[info] Python: 3.7 Pandas: 0.25.3 PyArrow: 0.14.0
```
Closes#26538 from HyukjinKwon/investigate-flaky-test.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Add ShowTableStatement and make SHOW TABLE EXTENDED go through the same catalog/table resolution framework of v2 commands.
We don’t have this methods in the catalog to implement an V2 command
- catalog.getPartition
- catalog.getTempViewOrPermanentTableMetadata
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing
```sql
USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW TABLE EXTENDED FROM LIKE 't' // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
Yes. When running SHOW TABLE EXTENDED Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26540 from planga82/feature/SPARK-29481_ShowTableExtended.
Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This is a follow-up pr to fix the code coming from #23400; it replaces `update` with `setByte` for ByteType in `JdbcUtils.makeGetter`.
### Why are the changes needed?
For better code.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#26532 from maropu/SPARK-26499-FOLLOWUP.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
In order to avoid frequently changing the value of `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions`, we usually set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` much larger than `spark.sql.shuffle.partitions` after enabling adaptive execution, which causes some bucket map join lose efficacy and add more `ShuffleExchange`.
How to reproduce:
```scala
val bucketedTableName = "bucketed_table"
spark.range(10000).write.bucketBy(500, "id").sortBy("id").mode(org.apache.spark.sql.SaveMode.Overwrite).saveAsTable(bucketedTableName)
val bucketedTable = spark.table(bucketedTableName)
val df = spark.range(8)
spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1)
// Spark 2.4. spark.sql.adaptive.enabled=false
// We set spark.sql.shuffle.partitions <= 500 every time based on our data in this case.
spark.conf.set("spark.sql.shuffle.partitions", 500)
bucketedTable.join(df, "id").explain()
// Since 3.0. We enabled adaptive execution and set spark.sql.adaptive.shuffle.maxNumPostShufflePartitions to a larger values to fit more cases.
spark.conf.set("spark.sql.adaptive.enabled", true)
spark.conf.set("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions", 1000)
bucketedTable.join(df, "id").explain()
```
```
scala> bucketedTable.join(df, "id").explain()
== Physical Plan ==
*(4) Project [id#5L]
+- *(4) SortMergeJoin [id#5L], [id#7L], Inner
:- *(1) Sort [id#5L ASC NULLS FIRST], false, 0
: +- *(1) Project [id#5L]
: +- *(1) Filter isnotnull(id#5L)
: +- *(1) ColumnarToRow
: +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 500 out of 500
+- *(3) Sort [id#7L ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(id#7L, 500), true, [id=#49]
+- *(2) Range (0, 8, step=1, splits=16)
```
vs
```
scala> bucketedTable.join(df, "id").explain()
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- Project [id#5L]
+- SortMergeJoin [id#5L], [id#7L], Inner
:- Sort [id#5L ASC NULLS FIRST], false, 0
: +- Exchange hashpartitioning(id#5L, 1000), true, [id=#93]
: +- Project [id#5L]
: +- Filter isnotnull(id#5L)
: +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 500 out of 500
+- Sort [id#7L ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(id#7L, 1000), true, [id=#92]
+- Range (0, 8, step=1, splits=16)
```
This PR makes read bucketed tables always obeys `spark.sql.shuffle.partitions` even enabling adaptive execution and set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` to avoid add more `ShuffleExchange`.
### Why are the changes needed?
Do not degrade performance after enabling adaptive execution.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Unit test.
Closes#26409 from wangyum/SPARK-29655.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Upgrade Apache Arrow to version 0.15.1. This includes Java artifacts and increases the minimum required version of PyArrow also.
Version 0.12.0 to 0.15.1 includes the following selected fixes/improvements relevant to Spark users:
* ARROW-6898 - [Java] Fix potential memory leak in ArrowWriter and several test classes
* ARROW-6874 - [Python] Memory leak in Table.to_pandas() when conversion to object dtype
* ARROW-5579 - [Java] shade flatbuffer dependency
* ARROW-5843 - [Java] Improve the readability and performance of BitVectorHelper#getNullCount
* ARROW-5881 - [Java] Provide functionalities to efficiently determine if a validity buffer has completely 1 bits/0 bits
* ARROW-5893 - [C++] Remove arrow::Column class from C++ library
* ARROW-5970 - [Java] Provide pointer to Arrow buffer
* ARROW-6070 - [Java] Avoid creating new schema before IPC sending
* ARROW-6279 - [Python] Add Table.slice method or allow slices in \_\_getitem\_\_
* ARROW-6313 - [Format] Tracking for ensuring flatbuffer serialized values are aligned in stream/files.
* ARROW-6557 - [Python] Always return pandas.Series from Array/ChunkedArray.to_pandas, propagate field names to Series from RecordBatch, Table
* ARROW-2015 - [Java] Use Java Time and Date APIs instead of JodaTime
* ARROW-1261 - [Java] Add container type for Map logical type
* ARROW-1207 - [C++] Implement Map logical type
Changelog can be seen at https://arrow.apache.org/release/0.15.0.html
### Why are the changes needed?
Upgrade to get bug fixes, improvements, and maintain compatibility with future versions of PyArrow.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Existing tests, manually tested with Python 3.7, 3.8
Closes#26133 from BryanCutler/arrow-upgrade-015-SPARK-29376.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
move interval tests to `interval.sql`, and import it to `ansi/interval.sql`
### Why are the changes needed?
improve test coverage
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
N/A
Closes#26515 from cloud-fan/test.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/23977 I made a mistake related to this line: 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL112)
Previously,
1. the reader iterator for R worker read some initial data eagerly during RDD materialization. So it read the data before actual execution. For some reasons, in this case, it showed standard error from R worker.
2. After that, when error happens during actual execution, stderr wasn't shown: 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL260)
After my change 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL112), it now ignores 1. case and only does 2. of previous code path, because 1. does not happen anymore as I avoided to such eager execution (which is consistent with PySpark code path).
This PR proposes to do only 1. before/after execution always because It is pretty much possible R worker was failed during actual execution and it's best to show the stderr from R worker whenever possible.
### Why are the changes needed?
It currently swallows standard error from R worker which makes debugging harder.
### Does this PR introduce any user-facing change?
Yes,
```R
df <- createDataFrame(list(list(n=1)))
collect(dapply(df, function(x) {
stop("asdkjasdjkbadskjbsdajbk")
x
}, structType("a double")))
```
**Before:**
```
Error in handleErrors(returnStatus, conn) :
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 13.0 failed 1 times, most recent failure: Lost task 0.0 in stage 13.0 (TID 13, 192.168.35.193, executor driver): org.apache.spark.SparkException: R worker exited unexpectedly (cranshed)
at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:130)
at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:118)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:726)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:337)
at org.apache.spark.
```
**After:**
```
Error in handleErrors(returnStatus, conn) :
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.35.193, executor driver): org.apache.spark.SparkException: R unexpectedly exited.
R worker produced errors: Error in computeFunc(inputData) : asdkjasdjkbadskjbsdajbk
at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:144)
at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:137)
at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38)
at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:128)
at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:113)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegen
```
### How was this patch tested?
Manually tested and unittest was added.
Closes#26517 from HyukjinKwon/SPARK-26923-followup.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR addresses issues where conflicting attributes in `Expand` are not correctly handled.
### Why are the changes needed?
```Scala
val numsDF = Seq(1, 2, 3, 4, 5, 6).toDF("nums")
val cubeDF = numsDF.cube("nums").agg(max(lit(0)).as("agcol"))
cubeDF.join(cubeDF, "nums").show
```
fails with the following exception:
```
org.apache.spark.sql.AnalysisException:
Failure when resolving conflicting references in Join:
'Join Inner
:- Aggregate [nums#38, spark_grouping_id#36], [nums#38, max(0) AS agcol#35]
: +- Expand [List(nums#3, nums#37, 0), List(nums#3, null, 1)], [nums#3, nums#38, spark_grouping_id#36]
: +- Project [nums#3, nums#3 AS nums#37]
: +- Project [value#1 AS nums#3]
: +- LocalRelation [value#1]
+- Aggregate [nums#38, spark_grouping_id#36], [nums#38, max(0) AS agcol#58]
+- Expand [List(nums#3, nums#37, 0), List(nums#3, null, 1)], [nums#3, nums#38, spark_grouping_id#36]
^^^^^^^
+- Project [nums#3, nums#3 AS nums#37]
+- Project [value#1 AS nums#3]
+- LocalRelation [value#1]
Conflicting attributes: nums#38
```
As you can see from the above plan, `num#38`, the output of `Expand` on the right side of `Join`, should have been handled to produce new attribute. Since the conflict is not resolved in `Expand`, the failure is happening upstream at `Aggregate`. This PR addresses handling conflicting attributes in `Expand`.
### Does this PR introduce any user-facing change?
Yes, the previous example now shows the following output:
```
+----+-----+-----+
|nums|agcol|agcol|
+----+-----+-----+
| 1| 0| 0|
| 6| 0| 0|
| 4| 0| 0|
| 2| 0| 0|
| 5| 0| 0|
| 3| 0| 0|
+----+-----+-----+
```
### How was this patch tested?
Added new unit test.
Closes#26441 from imback82/spark-29682.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This pr is to support `--import` directive to load queries from another test case in SQLQueryTestSuite.
This fix comes from the cloud-fan suggestion in https://github.com/apache/spark/pull/26479#discussion_r345086978
### Why are the changes needed?
This functionality might reduce duplicate test code in `SQLQueryTestSuite`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Run `SQLQueryTestSuite`.
Closes#26497 from maropu/ImportTests.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Make SparkSQL's `cast to boolean` behavior be consistent with PostgreSQL when
spark.sql.dialect is configured as PostgreSQL.
### Why are the changes needed?
SparkSQL and PostgreSQL have a lot different cast behavior between types by default. We should make SparkSQL's cast behavior be consistent with PostgreSQL when `spark.sql.dialect` is configured as PostgreSQL.
### Does this PR introduce any user-facing change?
Yes. If user switches to PostgreSQL dialect now, they will
* get an exception if they input a invalid string, e.g "erut", while they get `null` before;
* get an exception if they input `TimestampType`, `DateType`, `LongType`, `ShortType`, `ByteType`, `DecimalType`, `DoubleType`, `FloatType` values, while they get `true` or `false` result before.
And here're evidences for those unsupported types from PostgreSQL:
timestamp:
```
postgres=# select cast(cast('2019-11-11' as timestamp) as boolean);
ERROR: cannot cast type timestamp without time zone to boolean
```
date:
```
postgres=# select cast(cast('2019-11-11' as date) as boolean);
ERROR: cannot cast type date to boolean
```
bigint:
```
postgres=# select cast(cast('20191111' as bigint) as boolean);
ERROR: cannot cast type bigint to boolean
```
smallint:
```
postgres=# select cast(cast(2019 as smallint) as boolean);
ERROR: cannot cast type smallint to boolean
```
bytea:
```
postgres=# select cast(cast('2019' as bytea) as boolean);
ERROR: cannot cast type bytea to boolean
```
decimal:
```
postgres=# select cast(cast('2019' as decimal) as boolean);
ERROR: cannot cast type numeric to boolean
```
float:
```
postgres=# select cast(cast('2019' as float) as boolean);
ERROR: cannot cast type double precision to boolean
```
### How was this patch tested?
Added and tested manually.
Closes#26463 from Ngone51/dev-postgre-cast2bool.
Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
We already know task attempts that do not clean up output files in staging directory can cause job failure (SPARK-27194). There was proposals trying to fix it by changing output filename, or deleting existing output files. These proposals are not reliable completely.
The difficulty is, as previous failed task attempt wrote the output file, at next task attempt the output file is still under same staging directory, even the output file name is different.
If the job will go to fail eventually, there is no point to re-run the task until max attempts are reached. For the jobs running a lot of time, re-running the task can waste a lot of time.
This patch proposes to let Spark detect such file already exist exception and stop the task set early.
### Why are the changes needed?
For now, if FileAlreadyExistsException is thrown during data writing job in SQL, the job will continue re-running task attempts until max failure number is reached. It is no point for re-running tasks as task attempts will also fail because they can not write to the existing file too. We should stop the task set early.
### Does this PR introduce any user-facing change?
Yes. If FileAlreadyExistsException is thrown during data writing job in SQL, no more task attempts are re-tried and the task set will be stoped early.
### How was this patch tested?
Unit test.
Closes#26312 from viirya/stop-taskset-if-outputfile-exists.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType as setShort() and setByte(). Changes in JDBCUtils.scala
Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType.
#### Problems
- In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively.
```
case ShortType =>
(stmt: PreparedStatement, row: Row, pos: Int) =>
stmt.setInt(pos + 1, row.getShort(pos))
The issue was pointed out by maropu
case ByteType =>
(stmt: PreparedStatement, row: Row, pos: Int) =>
stmt.setInt(pos + 1, row.getByte(pos))
```
- Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType()
``` case java.sql.Types.TINYINT => IntegerType ```
- At line 172 ShortType was wrongly interpreted as IntegerType
``` case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) ```
- All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes.
### Why are the changes needed?
A given type should be set using the right type.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Corrected Unit test cases where applicable. Validated in CI/CD
Added a test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows.
```
./build/mvn install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12
```
Closes#26301 from shivsood/shorttype_fix_maropu.
Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
`saveAsTable` had an oversight where write options were not considered in the append save mode.
### Why are the changes needed?
Address the bug so that write options can be considered during appends.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Unit test added that looks in the logic plan of `AppendData` for the existing write options.
Closes#26474 from SpaceRangerWes/master.
Authored-by: Wesley Hoffman <wesleyhoffman109@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR adds a SQL Conf: `spark.sql.streaming.stopActiveRunOnRestart`. When this conf is `true` (by default it is), an already running stream will be stopped, if a new copy gets launched on the same checkpoint location.
### Why are the changes needed?
In multi-tenant environments where you have multiple SparkSessions, you can accidentally start multiple copies of the same stream (i.e. streams using the same checkpoint location). This will cause all new instantiations of the new stream to fail. However, sometimes you may want to turn off the old stream, as the old stream may have turned into a zombie (you no longer have access to the query handle or SparkSession).
It would be nice to have a SQL flag that allows the stopping of the old stream for such zombie cases.
### Does this PR introduce any user-facing change?
Yes. Now by default, if you launch a new copy of an already running stream on a multi-tenant cluster, the existing stream will be stopped.
### How was this patch tested?
Unit tests in StreamingQueryManagerSuite
Closes#26225 from brkyvz/stopStream.
Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
### What changes were proposed in this pull request?
rename EveryAgg/AnyAgg to BoolAnd/BoolOr
### Why are the changes needed?
Under ansi mode, `every`, `any` and `some` are reserved keywords and can't be used as function names. `EveryAgg`/`AnyAgg` has several aliases and I think it's better to not pick reserved keywords as the primary name.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26486 from cloud-fan/naming.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
rename the config to address the comment: https://github.com/apache/spark/pull/24594#discussion_r285431212
improve the config description, provide a default value to simplify the code.
### Why are the changes needed?
make the config more understandable.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26395 from cloud-fan/config.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The current parse and analyze flow for DELETE is: 1, the SQL string will be firstly parsed to `DeleteFromStatement`; 2, the `DeleteFromStatement` be converted to `DeleteFromTable`. However, the SQL string can be parsed to `DeleteFromTable` directly, where a `DeleteFromStatement` seems to be redundant.
It is the same for UPDATE.
This pr removes the unnecessary `DeleteFromStatement` and `UpdateTableStatement`.
### Why are the changes needed?
This makes the codes for DELETE and UPDATE cleaner, and keep align with MERGE INTO.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existed tests and new tests.
Closes#26464 from xianyinxin/SPARK-29835.
Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Currently, `SupportsNamespaces.dropNamespace` drops a namespace only if it is empty. Thus, to implement a cascading drop, one needs to iterate all objects (tables, view, etc.) within the namespace (including its sub-namespaces recursively) and drop them one by one. This can have a negative impact on the performance when there are large number of objects.
Instead, this PR proposes to change the default behavior of dropping a namespace to cascading such that implementing cascading/non-cascading drop is simpler without performance penalties.
### Why are the changes needed?
The new behavior makes implementing cascading/non-cascading drop simple without performance penalties.
### Does this PR introduce any user-facing change?
Yes. The default behavior of `SupportsNamespaces.dropNamespace` is now cascading.
### How was this patch tested?
Added new unit tests.
Closes#26476 from imback82/drop_ns_cascade.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add 3 interval functions justify_days, justify_hours, justif_interval to support justify interval values
### Why are the changes needed?
For feature parity with postgres
add three interval functions to justify interval values.
justify_days(interval) | interval | Adjust interval so 30-day time periods are represented as months | justify_days(interval '35 days') | 1 mon 5 days
-- | -- | -- | -- | --
justify_hours(interval) | interval | Adjust interval so 24-hour time periods are represented as days | justify_hours(interval '27 hours') | 1 day 03:00:00
justify_interval(interval) | interval | Adjust interval using justify_days and justify_hours, with additional sign adjustments | justify_interval(interval '1 mon -1 hour') | 29 days 23:00:00
### Does this PR introduce any user-facing change?
yes. new interval functions are added
### How was this patch tested?
add ut
Closes#26465 from yaooqinn/SPARK-29390.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Follow comment of https://github.com/apache/spark/pull/25854#discussion_r342383272
### Why are the changes needed?
NO
### Does this PR introduce any user-facing change?
NO
### How was this patch tested?
ADD TEST CASE
Closes#26406 from AngersZhuuuu/SPARK-29145-FOLLOWUP.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
All tooltips message will display in centre.
### Why are the changes needed?
Some time tooltips will hide the data of column and tooltips display position will be inconsistent in UI.
### Does this PR introduce any user-facing change?
yes.
![Screenshot 2019-10-26 at 3 08 51 AM](https://user-images.githubusercontent.com/8948111/67606124-04dd0d80-f79e-11e9-865a-b7e9bffc9890.png)
### How was this patch tested?
Manual test.
Closes#26263 from 07ARB/SPARK-29570.
Lead-authored-by: Ankitraj <8948111+07ARB@users.noreply.github.com>
Co-authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
When creating v2 expressions, we have public java APIs, as well as interval scala APIs. All of these APIs take a string column name and parse it to `NamedReference`.
This is convenient for end-users, but not for interval development. For example, the query plan already contains the parsed partition/bucket column names, and it's tricky if we need to quote the names before creating v2 expressions.
This PR proposes to change the interval scala APIs to take `NamedReference` directly, with a new method to create `NamedReference` with the exact name parts. The public java APIs are not changed.
### Why are the changes needed?
fix a bug, and make it easier to create v2 expressions correctly in the future.
### Does this PR introduce any user-facing change?
yes, now v2 CREATE TABLE works as expected.
### How was this patch tested?
a new test
Closes#26425 from cloud-fan/extract.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Ryan Blue <blue@apache.org>
### What changes were proposed in this pull request?
When whole stage codegen `HashAggregateExec`, create the hash map when we begin to process inputs.
### Why are the changes needed?
Sort-merge join completes directly if the left side table is empty. If there is an aggregate in the right side, the aggregate will not be triggered at all, but its hash map is created during codegen and can't be released.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
a new test
Closes#26471 from cloud-fan/memory.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
```sql
-- !query 83
select -integer '7'
-- !query 83 schema
struct<7:int>
-- !query 83 output
7
-- !query 86
select -date '1999-01-01'
-- !query 86 schema
struct<DATE '1999-01-01':date>
-- !query 86 output
1999-01-01
-- !query 87
select -timestamp '1999-01-01'
-- !query 87 schema
struct<TIMESTAMP('1999-01-01 00:00:00'):timestamp>
-- !query 87 output
1999-01-01 00:00:00
```
the integer should be -7 and the date and timestamp results are confusing which should throw exceptions
### Why are the changes needed?
bug fix
### Does this PR introduce any user-facing change?
NO
### How was this patch tested?
ADD UTs
Closes#26479 from yaooqinn/SPARK-29855.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Add ShowTablePropertiesStatement and make SHOW TBLPROPERTIES go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW TBLPROPERTIES t // report table not found as there is no table t in the session catalog
### Does this PR introduce any user-facing change?
yes. When running SHOW TBLPROPERTIES Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26176 from planga82/feature/SPARK-29519_SHOW_TBLPROPERTIES_datasourceV2.
Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This patch fixes the edge case of streaming left/right outer join described below:
Suppose query is provided as
`select * from A join B on A.id = B.id AND (A.ts <= B.ts AND B.ts <= A.ts + interval 5 seconds)`
and there're two rows for L1 (from A) and R1 (from B) which ensures L1.id = R1.id and L1.ts = R1.ts.
(we can simply imagine it from self-join)
Then Spark processes L1 and R1 as below:
- row L1 and row R1 are joined at batch 1
- row R1 is evicted at batch 2 due to join and watermark condition, whereas row L1 is not evicted
- row L1 is evicted at batch 3 due to join and watermark condition
When determining outer rows to match with null, Spark applies some assumption commented in codebase, as below:
```
Checking whether the current row matches a key in the right side state, and that key
has any value which satisfies the filter function when joined. If it doesn't,
we know we can join with null, since there was never (including this batch) a match
within the watermark period. If it does, there must have been a match at some point, so
we know we can't join with null.
```
But as explained the edge-case earlier, the assumption is not correct. As we don't have any good assumption to optimize which doesn't have edge-case, we have to track whether such row is matched with others before, and match with null row only when the row is not matched.
To track the matching of row, the patch adds a new state to streaming join state manager, and mark whether the row is matched to others or not. We leverage the information when dealing with eviction of rows which would be candidates to match with null rows.
This approach introduces new state format which is not compatible with old state format - queries with old state format will be still running but they will still have the issue and be required to discard checkpoint and rerun to take this patch in effect.
### Why are the changes needed?
This patch fixes a correctness issue.
### Does this PR introduce any user-facing change?
No for compatibility viewpoint, but we'll encourage end users to discard the old checkpoint and rerun the query if they run stream-stream outer join query with old checkpoint, which might be "yes" for the question.
### How was this patch tested?
Added UT which fails on current Spark and passes with this patch. Also passed existing streaming join UTs.
Closes#26108 from HeartSaVioR/SPARK-26154-shorten-alternative.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
This unblocks the event handling thread, which should help avoid dropped
events when large queries are running.
Existing unit tests should already cover this code.
Closes#26405 from vanzin/SPARK-29766.
Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
For better test coverage, this pr is to add join-related configs in `inner-join.sql` and `postgreSQL/join.sql`. These join related configs were just copied from ones in the other join-related tests in `SQLQueryTestSuite` (e.g., https://github.com/apache/spark/blob/master/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql#L2-L4).
### Why are the changes needed?
Better test coverage.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#26459 from maropu/AddJoinConds.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
With the latest string to literal optimization https://github.com/apache/spark/pull/26256, some interval strings can not be cast when there are some spaces between signs and unit values. After state `PARSE_SIGN`, it directly goes to `PARSE_UNIT_VALUE` when takes a space character as the end. So when there are some white spaces come before the real unit value, it fails to parse, we should add a new state like `TRIM_VALUE` to trim all these spaces.
How to re-produce, which aim the revisions since https://github.com/apache/spark/pull/26256 is merged
```sql
select cast(v as interval) from values ('+ 1 second') t(v);
select cast(v as interval) from values ('- 1 second') t(v);
```
### Why are the changes needed?
bug fix
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
1. ut
2. new benchmark test
Closes#26449 from yaooqinn/SPARK-29605.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Hive support STORED AS new file format syntax:
```sql
CREATE TABLE tbl(a int) STORED AS TEXTFILE;
CREATE TABLE tbl2 LIKE tbl STORED AS PARQUET;
```
We add a similar syntax for Spark. Here we separate to two features:
1. specify a different table provider in CREATE TABLE LIKE
2. Hive compatibility
In this PR, we address the first one:
- [ ] Using `USING provider` to specify a different table provider in CREATE TABLE LIKE.
- [ ] Using `STORED AS file_format` in CREATE TABLE LIKE to address Hive compatibility.
### Why are the changes needed?
Use CREATE TABLE tb1 LIKE tb2 command to create an empty table tb1 based on the definition of table tb2. The most user case is to create tb1 with the same schema of tb2. But an inconvenient case here is this command also copies the FileFormat from tb2, it cannot change the input/output format and serde. Add the ability of changing file format is useful for some scenarios like upgrading a table from a low performance file format to a high performance one (parquet, orc).
### Does this PR introduce any user-facing change?
Add a new syntax based on current CTL:
```sql
CREATE TABLE tbl2 LIKE tbl [USING parquet];
```
### How was this patch tested?
Modify some exist UTs.
Closes#26097 from LantaoJin/SPARK-29421.
Authored-by: lajin <lajin@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose new expression `MakeInterval` and register it as the function `make_interval`. The function accepts the following parameters:
- `years` - the number of years in the interval, positive or negative. The parameter is multiplied by 12, and added to interval's `months`.
- `months` - the number of months in the interval, positive or negative.
- `weeks` - the number of months in the interval, positive or negative. The parameter is multiplied by 7, and added to interval's `days`.
- `hours`, `mins` - the number of hours and minutes. The parameters can be negative or positive. They are converted to microseconds and added to interval's `microseconds`.
- `seconds` - the number of seconds with the fractional part in microseconds precision. It is converted to microseconds, and added to total interval's `microseconds` as `hours` and `minutes`.
For example:
```sql
spark-sql> select make_interval(2019, 11, 1, 1, 12, 30, 01.001001);
2019 years 11 months 8 days 12 hours 30 minutes 1.001001 seconds
```
### Why are the changes needed?
- To improve user experience with Spark SQL, and allow users making `INTERVAL` columns from other columns containing `years`, `months` ... `seconds`. Currently, users can make an `INTERVAL` column from other columns only by constructing a `STRING` column and cast it to `INTERVAL`. Have a look at the `IntervalBenchmark` as an example.
- To maintain feature parity with PostgreSQL which provides such function:
```sql
# SELECT make_interval(2019, 11);
make_interval
--------------------
2019 years 11 mons
```
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
- By new tests for the `MakeInterval` expression to `IntervalExpressionsSuite`
- By tests in `interval.sql`
Closes#26446 from MaxGekk/make_interval.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
- `SqlBase.g4` is modified to support a negative sign `-` in the interval type constructor from a string and in interval literals
- Negate interval in `AstBuilder` if a sign presents.
- Interval related SQL statements are moved from `inputs/datetime.sql` to new file `inputs/interval.sql`
For example:
```sql
spark-sql> select -interval '-1 month 1 day -1 second';
1 months -1 days 1 seconds
spark-sql> select -interval -1 month 1 day -1 second;
1 months -1 days 1 seconds
```
### Why are the changes needed?
For feature parity with PostgreSQL which supports that:
```sql
# select -interval '-1 month 1 day -1 second';
?column?
-------------------------
1 mon -1 days +00:00:01
(1 row)
```
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
- Added tests to `ExpressionParserSuite`
- by `interval.sql`
Closes#26438 from MaxGekk/negative-interval.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
Add AlterViewAsStatement and make ALTER VIEW ... QUERY go through the same catalog/table resolution framework of v2 commands.
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC v // success and describe the view v from my_catalog
ALTER VIEW v SELECT 1 // report view not found as there is no view v in the session catalog
```
Yes. When running ALTER VIEW ... QUERY, Spark fails the command if the current catalog is set to a v2 catalog, or the view name specified a v2 catalog.
unit tests
Closes#26453 from huaxingao/spark-29730.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR supports MERGE INTO in the parser and add the corresponding logical plan. The SQL syntax likes,
```
MERGE INTO [ds_catalog.][multi_part_namespaces.]target_table [AS target_alias]
USING [ds_catalog.][multi_part_namespaces.]source_table | subquery [AS source_alias]
ON <merge_condition>
[ WHEN MATCHED [ AND <condition> ] THEN <matched_action> ]
[ WHEN MATCHED [ AND <condition> ] THEN <matched_action> ]
[ WHEN NOT MATCHED [ AND <condition> ] THEN <not_matched_action> ]
```
where
```
<matched_action> =
DELETE |
UPDATE SET * |
UPDATE SET column1 = value1 [, column2 = value2 ...]
<not_matched_action> =
INSERT * |
INSERT (column1 [, column2 ...]) VALUES (value1 [, value2 ...])
```
### Why are the changes needed?
This is a start work for introduce `MERGE INTO` support for the builtin datasource, and the design work for the `MERGE INTO` support in DSV2.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
New test cases.
Closes#26167 from xianyinxin/SPARK-28893.
Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Replace qualifiedName with multipartIdentifier in parser rules of DDL commands.
### Why are the changes needed?
There are identifiers in some DDL rules we use `qualifiedName`. We should use `multipartIdentifier` because it can capture wrong identifiers such as `test-table`, `test-col`.
### Does this PR introduce any user-facing change?
Yes. Wrong identifiers such as test-table, will be captured now after this change.
### How was this patch tested?
Unit tests.
Closes#26419 from viirya/SPARK-29680-followup2.
Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
### What changes were proposed in this pull request?
interval type support >, >=, <, <=, =, <=>, order by, min,max..
### Why are the changes needed?
Part of SPARK-27764 Feature Parity between PostgreSQL and Spark
### Does this PR introduce any user-facing change?
yes, we now support compare intervals
### How was this patch tested?
add ut
Closes#26337 from yaooqinn/SPARK-29679.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
avg aggregate support interval type values
### Why are the changes needed?
Part of SPARK-27764 Feature Parity between PostgreSQL and Spark
### Does this PR introduce any user-facing change?
yes, we can do avg on intervals
### How was this patch tested?
add ut
Closes#26347 from yaooqinn/SPARK-29688.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
V2 catalog support namespace, we should add `withNamespace` like `withDatabase`.
### Why are the changes needed?
Make test easy.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Add UT.
Closes#26411 from ulysses-you/Add-test-with-namespace.
Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
```java
public static final int YEARS_PER_DECADE = 10;
public static final int YEARS_PER_CENTURY = 100;
public static final int YEARS_PER_MILLENNIUM = 1000;
public static final byte MONTHS_PER_QUARTER = 3;
public static final int MONTHS_PER_YEAR = 12;
public static final byte DAYS_PER_WEEK = 7;
public static final long DAYS_PER_MONTH = 30L;
public static final long HOURS_PER_DAY = 24L;
public static final long MINUTES_PER_HOUR = 60L;
public static final long SECONDS_PER_MINUTE = 60L;
public static final long SECONDS_PER_HOUR = MINUTES_PER_HOUR * SECONDS_PER_MINUTE;
public static final long SECONDS_PER_DAY = HOURS_PER_DAY * SECONDS_PER_HOUR;
public static final long MILLIS_PER_SECOND = 1000L;
public static final long MILLIS_PER_MINUTE = SECONDS_PER_MINUTE * MILLIS_PER_SECOND;
public static final long MILLIS_PER_HOUR = MINUTES_PER_HOUR * MILLIS_PER_MINUTE;
public static final long MILLIS_PER_DAY = HOURS_PER_DAY * MILLIS_PER_HOUR;
public static final long MICROS_PER_MILLIS = 1000L;
public static final long MICROS_PER_SECOND = MILLIS_PER_SECOND * MICROS_PER_MILLIS;
public static final long MICROS_PER_MINUTE = SECONDS_PER_MINUTE * MICROS_PER_SECOND;
public static final long MICROS_PER_HOUR = MINUTES_PER_HOUR * MICROS_PER_MINUTE;
public static final long MICROS_PER_DAY = HOURS_PER_DAY * MICROS_PER_HOUR;
public static final long MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY;
/* 365.25 days per year assumes leap year every four years */
public static final long MICROS_PER_YEAR = (36525L * MICROS_PER_DAY) / 100;
public static final long NANOS_PER_MICROS = 1000L;
public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
```
The above parameters are defined in IntervalUtils, DateTimeUtils, and CalendarInterval, some of them are redundant, some of them are cross-referenced.
### Why are the changes needed?
To simplify code, enhance consistency and reduce risks
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
modified uts
Closes#26399 from yaooqinn/SPARK-29757.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
remove the leading "interval" in `CalendarInterval.toString`.
### Why are the changes needed?
Although it's allowed to have "interval" prefix when casting string to int, it's not recommended.
This is also consistent with pgsql:
```
cloud0fan=# select interval '1' day;
interval
----------
1 day
(1 row)
```
### Does this PR introduce any user-facing change?
yes, when display a dataframe with interval type column, the result is different.
### How was this patch tested?
updated tests.
Closes#26401 from cloud-fan/interval.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose new function `stringToInterval()` in `IntervalUtils` for converting `UTF8String` to `CalendarInterval`. The function is used in casting a `STRING` column to an `INTERVAL` column.
### Why are the changes needed?
The proposed implementation is ~10 times faster. For example, parsing 9 interval units on JDK 8:
Before:
```
9 units w/ interval 14004 14125 116 0.1 14003.6 0.0X
9 units w/o interval 13785 14056 290 0.1 13784.9 0.0X
```
After:
```
9 units w/ interval 1343 1344 1 0.7 1343.0 0.3X
9 units w/o interval 1345 1349 8 0.7 1344.6 0.3X
```
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
- By new tests for `stringToInterval` in `IntervalUtilsSuite`
- By existing tests
Closes#26256 from MaxGekk/string-to-interval.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Handle the inconsistence dividing zeros between literals and columns.
fix the null issue too.
### Why are the changes needed?
BUG FIX
### 1 Handle the inconsistence dividing zeros between literals and columns
```sql
-- !query 24
select
k,
v,
cast(k as interval) / v,
cast(k as interval) * v
from VALUES
('1 seconds', 1),
('2 seconds', 0),
('3 seconds', null),
(null, null),
(null, 0) t(k, v)
-- !query 24 schema
struct<k:string,v:int,divide_interval(CAST(k AS INTERVAL), CAST(v AS DOUBLE)):interval,multiply_interval(CAST(k AS INTERVAL), CAST(v AS DOUBLE)):interval>
-- !query 24 output
1 seconds 1 interval 1 seconds interval 1 seconds
2 seconds 0 interval 0 microseconds interval 0 microseconds
3 seconds NULL NULL NULL
NULL 0 NULL NULL
NULL NULL NULL NULL
```
```sql
-- !query 21
select interval '1 year 2 month' / 0
-- !query 21 schema
struct<divide_interval(interval 1 years 2 months, CAST(0 AS DOUBLE)):interval>
-- !query 21 output
NULL
```
in the first case, interval ’2 seconds ‘ / 0, it produces `interval 0 microseconds `
in the second case, it is `null`
### 2 null literal issues
```sql
-- !query 20
select interval '1 year 2 month' / null
-- !query 20 schema
struct<>
-- !query 20 output
org.apache.spark.sql.AnalysisException
cannot resolve '(interval 1 years 2 months / NULL)' due to data type mismatch: differing types in '(interval 1 years 2 months / NULL)' (interval and null).; line 1 pos 7
-- !query 22
select interval '4 months 2 weeks 6 days' * null
-- !query 22 schema
struct<>
-- !query 22 output
org.apache.spark.sql.AnalysisException
cannot resolve '(interval 4 months 20 days * NULL)' due to data type mismatch: differing types in '(interval 4 months 20 days * NULL)' (interval and null).; line 1 pos 7
-- !query 23
select null * interval '4 months 2 weeks 6 days'
-- !query 23 schema
struct<>
-- !query 23 output
org.apache.spark.sql.AnalysisException
cannot resolve '(NULL * interval 4 months 20 days)' due to data type mismatch: differing types in '(NULL * interval 4 months 20 days)' (null and interval).; line 1 pos 7
```
dividing or multiplying null literals, error occurs; where in column is fine as the first case
### Does this PR introduce any user-facing change?
NO, maybe yes, but it is just a follow-up
### How was this patch tested?
add uts
cc cloud-fan MaxGekk maropu
Closes#26410 from yaooqinn/SPARK-29387.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Update `LocalShuffleReaderExec.outputPartitioning` to use attributes from `ReusedQueryStage`.
This also removes the override `doCanonicalize` in local/coalesced shuffle reader, as these 2 operators change the output partitioning. It's not safe to strip them in the canonicalized query plan.
### Why are the changes needed?
We will have an invalid output partitioning if we don fix it.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26400 from cloud-fan/aqe.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
### What changes were proposed in this pull request?
This patch fixes the bug that `ContinuousMemoryStream[String]` throws error regarding ClassCastException - cast String to UTFString. This is because ContinuousMemoryStream and ContinuousRecordEndpoint uses origin input as it is for underlying data structure of Row, and encoding is missing here.
To force encoding, this patch changes the element type of underlying array to UnsafeRow instead of Any for ContinuousRecordEndpoint - ContinuousMemoryStream and TextSocketContinuousStream are modified to reflect the change.
### Why are the changes needed?
Above section describes the bug.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Add new UT to check for availability on couple of types.
Closes#26300 from HeartSaVioR/SPARK-29642.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
### What changes were proposed in this pull request?
instead of checking the exact number of local shuffle readers, we should check whether the number of shuffles is equal to the number of local readers.
### Why are the changes needed?
AQE is known to have randomness. We may pick different build side for broadcast join depending on which query stage finishes first. The decision to build side may add/remove shuffles downstream, so it's flaky to check the exact number of local shuffle readers.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
test only PR.
Closes#26394 from cloud-fan/test.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
### What changes were proposed in this pull request?
During creation of array, if CreateArray does not gets any children to set data type for array, it will create an array of null type .
### Why are the changes needed?
When empty array is created, it should be declared as array<null>.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Tested manually
Closes#26324 from amanomer/29462.
Authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This patch removes v1 ALTER TABLE CHANGE COLUMN syntax.
### Why are the changes needed?
Since in v2 we have ALTER TABLE CHANGE COLUMN and ALTER TABLE RENAME COLUMN, this old syntax is not necessary now and can be confusing.
The v2 ALTER TABLE CHANGE COLUMN should fallback to v1 AlterTableChangeColumnCommand (#26354).
### Does this PR introduce any user-facing change?
Yes, the old v1 ALTER TABLE CHANGE COLUMN syntax is removed.
### How was this patch tested?
Unit tests.
Closes#26338 from viirya/SPARK-29680.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Added new expressions `MultiplyInterval` and `DivideInterval` to multiply/divide an interval by a numeric. Updated `TypeCoercion.DateTimeOperations` to turn the `Multiply`/`Divide` expressions of `CalendarIntervalType` and `NumericType` to `MultiplyInterval`/`DivideInterval`.
To support new operations, added new methods `multiply()` and `divide()` to `CalendarInterval`.
### Why are the changes needed?
- To maintain feature parity with PostgreSQL which supports multiplication and division of intervals by doubles:
```sql
# select interval '1 hour' / double precision '1.5';
?column?
----------
00:40:00
```
- To conform the SQL standard which defines those operations: `numeric * interval`, `interval * numeric` and `interval / numeric`. See [4.5.3 Operations involving datetimes and intervals](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt).
- Improve Spark SQL UX and allow users to adjust interval columns. For example:
```sql
spark-sql> select (timestamp'now' - timestamp'yesterday') * 1.3;
interval 2 days 10 hours 39 minutes 38 seconds 568 milliseconds 900 microseconds
```
### Does this PR introduce any user-facing change?
Yes, previously the following query fails with the error:
```sql
spark-sql> select interval 1 hour 30 minutes * 1.5;
Error in query: cannot resolve '(interval 1 hours 30 minutes * 1.5BD)' due to data type mismatch: differing types in '(interval 1 hours 30 minutes * 1.5BD)' (interval and decimal(2,1)).; line 1 pos 7;
```
After:
```sql
spark-sql> select interval 1 hour 30 minutes * 1.5;
interval 2 hours 15 minutes
```
### How was this patch tested?
- Added tests for the `multiply()` and `divide()` methods to `CalendarIntervalSuite.java`
- New test suite `IntervalExpressionsSuite`
- by tests for `Multiply` -> `MultiplyInterval` and `Divide` -> `DivideInterval` in `TypeCoercionSuite`
- updated `datetime.sql`
Closes#26132 from MaxGekk/interval-mul-div.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add AlterTableSerDePropertiesStatement and make ALTER TABLE ... SET SERDE/SERDEPROPERTIES go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ALTER TABLE t SET SERDE 'org.apache.class' // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
Yes. When running ALTER TABLE ... SET SERDE/SERDEPROPERTIES, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26374 from huaxingao/spark_29695.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Disallow creating a permanent view that references a temporary view in **expressions**.
### Why are the changes needed?
Creating a permanent view that references a temporary view is currently disallowed. For example,
```SQL
# The following throws org.apache.spark.sql.AnalysisException
# Not allowed to create a permanent view `per_view` by referencing a temporary view `tmp`;
CREATE VIEW per_view AS SELECT t1.a, t2.b FROM base_table t1, (SELECT * FROM tmp) t2"
```
However, the following is allowed.
```SQL
CREATE VIEW per_view AS SELECT * FROM base_table WHERE EXISTS (SELECT * FROM tmp);
```
This PR fixes the bug where temporary views used inside expressions are not checked.
### Does this PR introduce any user-facing change?
Yes. Now the following SQL query throws an exception as expected:
```SQL
# The following throws org.apache.spark.sql.AnalysisException
# Not allowed to create a permanent view `per_view` by referencing a temporary view `tmp`;
CREATE VIEW per_view AS SELECT * FROM base_table WHERE EXISTS (SELECT * FROM tmp);
```
### How was this patch tested?
Added new unit tests.
Closes#26361 from imback82/spark-29630.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR introduces a new SQL command: `SHOW CURRENT NAMESPACE`.
### Why are the changes needed?
Datasource V2 supports multiple catalogs/namespaces and having `SHOW CURRENT NAMESPACE` to retrieve the current catalog/namespace info would be useful.
### Does this PR introduce any user-facing change?
Yes, the user can perform the following:
```
scala> spark.sql("SHOW CURRENT NAMESPACE").show
+-------------+---------+
| catalog|namespace|
+-------------+---------+
|spark_catalog| default|
+-------------+---------+
scala> spark.sql("USE testcat.ns1.ns2").show
scala> spark.sql("SHOW CURRENT NAMESPACE").show
+-------+---------+
|catalog|namespace|
+-------+---------+
|testcat| ns1.ns2|
+-------+---------+
```
### How was this patch tested?
Added unit tests.
Closes#26379 from imback82/show_current_catalog.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
This patch adds the option to clean up files which are completed in previous batch.
`cleanSource` -> "archive" / "delete" / "off"
The default value is "off", which Spark will do nothing.
If "delete" is specified, Spark will simply delete input files. If "archive" is specified, Spark will require additional config `sourceArchiveDir` which will be used to move input files to there. When archiving (via move) the path of input files are retained to the archived paths as sub-path.
Note that it is only applied to "micro-batch", since for batch all input files must be kept to get same result across multiple query executions.
## How was this patch tested?
Added UT. Manual test against local disk as well as HDFS.
Closes#22952 from HeartSaVioR/SPARK-20568.
Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Co-authored-by: Jungtaek Lim <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
This pr proposes to be case insensitive when matching dialects via jdbc url prefix.
When I use jdbc url such as: ```jdbc: MySQL://localhost/db``` to query data through sparksql, the result is wrong, but MySQL supports such url writing.
because sparksql matches MySQLDialect by prefix ```jdbc:mysql```, so ```jdbc: MySQL``` is not matched with the correct dialect. Therefore, it should be case insensitive when identifying the corresponding dialect through jdbc url
https://issues.apache.org/jira/browse/SPARK-28552
## How was this patch tested?
UT.
Closes#25287 from teeyog/sql_dialect.
Lead-authored-by: yong.tian1 <yong.tian1@dmall.com>
Co-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: Chris Martin <chris@cmartinit.co.uk>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Kent Yao <yaooqinn@hotmail.com>
Co-authored-by: teeyog <teeyog@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
`SampleExec` has a bug that it sets `needCopyResult` to false as long as the `withReplacement` parameter is false. This causes problems if its child needs to copy the result, e.g. a join.
### Why are the changes needed?
to fix a correctness issue
### Does this PR introduce any user-facing change?
Yes, the result will be corrected.
### How was this patch tested?
a new test
Closes#26387 from cloud-fan/sample-bug.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
sum support interval values
### Why are the changes needed?
Part of SPARK-27764 Feature Parity between PostgreSQL and Spark
### Does this PR introduce any user-facing change?
yes, sum can evaluate intervals
### How was this patch tested?
add ut
Closes#26325 from yaooqinn/SPARK-29663.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add AlterTableAddPartitionStatement and make ALTER TABLE ... ADD PARTITION go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ALTER TABLE t ADD PARTITION (id=1) // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
Yes. When running ALTER TABLE ... ADD PARTITION, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests
Closes#26369 from imback82/spark-29678.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
If the resolved table is v1 table, AlterTableAlterColumnStatement fallbacks to v1 AlterTableChangeColumnCommand.
### Why are the changes needed?
To make the catalog/table lookup logic consistent.
### Does this PR introduce any user-facing change?
Yes, a ALTER TABLE ALTER COLUMN command previously fails on v1 tables. After this, it falls back to v1 AlterTableChangeColumnCommand.
### How was this patch tested?
Unit test.
Closes#26354 from viirya/SPARK-29353.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to changed `CalendarInterval.toString`:
- to skip the `week` unit
- to convert `milliseconds` and `microseconds` as the fractional part of the `seconds` unit.
### Why are the changes needed?
To improve readability.
### Does this PR introduce any user-facing change?
Yes
### How was this patch tested?
- By `CalendarIntervalSuite` and `IntervalUtilsSuite`
- `literals.sql`, `datetime.sql` and `interval.sql`
Closes#26367 from MaxGekk/interval-to-string-format.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This is somewhat a complement of https://github.com/apache/spark/pull/21853.
The `Sort` without `Limit` operator in `Join` subquery is useless, it's the same case in `GroupBy` when the aggregation function is order irrelevant, such as `count`, `sum`.
This PR try to remove this kind of `Sort` operator in `SQL Optimizer`.
### Why are the changes needed?
For example, `select count(1) from (select a from test1 order by a)` is equal to `select count(1) from (select a from test1)`.
'select * from (select a from test1 order by a) t1 join (select b from test2) t2 on t1.a = t2.b' is equal to `select * from (select a from test1) t1 join (select b from test2) t2 on t1.a = t2.b`.
Remove useless `Sort` operator can improve performance.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Adding new UT `RemoveSortInSubquerySuite.scala`
Closes#26011 from WangGuangxin/remove_sorts.
Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Support non-reversed keywords to be used in high order functions.
### Why are the changes needed?
the keywords are non-reversed.
### Does this PR introduce any user-facing change?
yes, all non-reversed keywords can be used in high order function correctly
### How was this patch tested?
add uts
Closes#26366 from yaooqinn/SPARK-29722.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
The `assertEquals` method of JUnit Assert requires the first parameter to be the expected value. In this PR, I propose to change the order of parameters when the expected value is passed as the second parameter.
### Why are the changes needed?
Wrong order of assert parameters confuses when the assert fails and the parameters have special string representation. For example:
```java
assertEquals(input1.add(input2), new CalendarInterval(5, 5, 367200000000L));
```
```
java.lang.AssertionError:
Expected :interval 5 months 5 days 101 hours
Actual :interval 5 months 5 days 102 hours
```
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By existing tests.
Closes#26377 from MaxGekk/fix-order-in-assert-equals.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
re-arrange the parser rules to make it clear that multiple unit TO unit statement like `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH` is not allowed.
### Why are the changes needed?
This is definitely an accident that we support such a weird syntax in the past. It's not supported by any other DBs and I can't think of any use case of it. Also no test covers this syntax in the current codebase.
### Does this PR introduce any user-facing change?
Yes, and a migration guide item is added.
### How was this patch tested?
new tests.
Closes#26285 from cloud-fan/syntax.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
###What changes were proposed in this pull request?
Add AlterTableDropPartitionStatement and make ALTER TABLE/VIEW ... DROP PARTITION go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ALTER TABLE t DROP PARTITION (id=1) // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
Yes. When running ALTER TABLE/VIEW ... DROP PARTITION, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26303 from huaxingao/spark-29643.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Current CalendarInterval has 2 fields: months and microseconds. This PR try to change it
to 3 fields: months, days and microseconds. This is because one logical day interval may
have different number of microseconds (daylight saving).
### Why are the changes needed?
One logical day interval may have different number of microseconds (daylight saving).
For example, in PST timezone, there will be 25 hours from 2019-11-2 12:00:00 to
2019-11-3 12:00:00
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
unit test and new added test cases
Closes#26134 from LinhongLiu/calendarinterval.
Authored-by: Liu,Linhong <liulinhong@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add AlterTableRenamePartitionStatement and make ALTER TABLE ... RENAME TO PARTITION go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ALTER TABLE t PARTITION (id=1) RENAME TO PARTITION (id=2) // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
Yes. When running ALTER TABLE ... RENAME TO PARTITION, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26350 from huaxingao/spark_29676.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
### What changes were proposed in this pull request?
Fix JDBC metrics counter data type. Related pull request [26109](https://github.com/apache/spark/pull/26109).
### Why are the changes needed?
Avoid overflow.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Exists UT.
Closes#26346 from ulysses-you/SPARK-29687.
Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Now we use JDBC api and set an Illegal isolationLevel option, spark will throw a `scala.MatchError`, it's not friendly to user. So we should add an IllegalArgumentException.
### Why are the changes needed?
Make exception friendly to user.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Add UT.
Closes#26334 from ulysses-you/SPARK-29675.
Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
Bring back https://github.com/apache/spark/pull/25955
### What changes were proposed in this pull request?
This adds a new rule, `V2ScanRelationPushDown`, to push filters and projections in to a new `DataSourceV2ScanRelation` in the optimizer. That scan is then used when converting to a physical scan node. The new relation correctly reports stats based on the scan.
To run scan pushdown before rules where stats are used, this adds a new optimizer override, `earlyScanPushDownRules` and a batch for early pushdown in the optimizer, before cost-based join reordering. The other early pushdown rule, `PruneFileSourcePartitions`, is moved into the early pushdown rule set.
This also moves pushdown helper methods from `DataSourceV2Strategy` into a util class.
### Why are the changes needed?
This is needed for DSv2 sources to supply stats for cost-based rules in the optimizer.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
This updates the implementation of stats from `DataSourceV2Relation` so tests will fail if stats are accessed before early pushdown for v2 relations.
Closes#26341 from cloud-fan/back.
Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
[PR#25295](https://github.com/apache/spark/pull/25295) already implement the rule of converting the shuffle reader to local reader for the `BroadcastHashJoin` in probe side. This PR support converting the shuffle reader to local reader in build side.
### Why are the changes needed?
Improve performance
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
existing unit tests
Closes#26289 from JkSelf/supportTwoSideLocalReader.
Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is code cleanup PR for https://github.com/apache/spark/pull/25600, aiming to remove an unnecessary condition and to correct a code comment.
### Why are the changes needed?
For code cleanup only.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Passed existing tests.
Closes#26328 from maryannxue/dpp-followup.
Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
- Added `getDuration()` to calculate interval duration in specified time units assuming provided days per months
- Added `isNegative()` which return `true` is the interval duration is less than 0
- Fix checking negative intervals by using `isNegative()` in structured streaming classes
- Fix checking of `year-months` intervals
### Why are the changes needed?
This fixes incorrect checking of negative intervals. An interval is negative when its duration is negative but not if interval's months **or** microseconds is negative. Also this fixes checking of `year-month` interval support because the `month` field could be negative.
### Does this PR introduce any user-facing change?
Should not
### How was this patch tested?
- Added tests for the `getDuration()` and `isNegative()` methods to `IntervalUtilsSuite`
- By existing SS tests
Closes#26177 from MaxGekk/interval-is-positive.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Update `AlterTableSetLocationStatement` to store `partitionSpec` and make `ALTER TABLE a.b.c PARTITION(...) SET LOCATION 'loc'` fail if `partitionSpec` is set with unsupported message.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ALTER TABLE t PARTITION(...) SET LOCATION 'loc' // report set location with partition spec is not supported.
```
### Does this PR introduce any user-facing change?
yes. When running ALTER TABLE (set partition location), Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
New unit tests
Closes#26304 from imback82/alter_table_partition_loc.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add ShowColumnsStatement and make SHOW COLUMNS go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW COLUMNS FROM t // report table not found as there is no table t in the session catalog
### Does this PR introduce any user-facing change?
yes. When running SHOW COLUMNS Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26182 from planga82/feature/SPARK-29523_SHOW_COLUMNS_datasourceV2.
Authored-by: Unknown <soypab@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to extract parsing of the seconds interval units to the private method `parseNanos` in `IntervalUtils` and modify the code to correctly parse the fractional part of the seconds unit of intervals in the cases:
- When the fractional part has less than 9 digits
- The seconds unit is negative
### Why are the changes needed?
The changes are needed to fix the issues:
```sql
spark-sql> select interval '10.123456 seconds';
interval 10 seconds 123 microseconds
```
The correct result must be `interval 10 seconds 123 milliseconds 456 microseconds`
```sql
spark-sql> select interval '-10.123456789 seconds';
interval -9 seconds -876 milliseconds -544 microseconds
```
but the whole interval should be negated, and the result must be `interval -10 seconds -123 milliseconds -456 microseconds`, taking into account the truncation to microseconds.
### Does this PR introduce any user-facing change?
Yes. After changes:
```sql
spark-sql> select interval '10.123456 seconds';
interval 10 seconds 123 milliseconds 456 microseconds
spark-sql> select interval '-10.123456789 seconds';
interval -10 seconds -123 milliseconds -456 microseconds
```
### How was this patch tested?
By existing and new tests in `ExpressionParserSuite`.
Closes#26313 from MaxGekk/fix-interval-nanos-parsing.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This adds a new rule, `V2ScanRelationPushDown`, to push filters and projections in to a new `DataSourceV2ScanRelation` in the optimizer. That scan is then used when converting to a physical scan node. The new relation correctly reports stats based on the scan.
To run scan pushdown before rules where stats are used, this adds a new optimizer override, `earlyScanPushDownRules` and a batch for early pushdown in the optimizer, before cost-based join reordering. The other early pushdown rule, `PruneFileSourcePartitions`, is moved into the early pushdown rule set.
This also moves pushdown helper methods from `DataSourceV2Strategy` into a util class.
### Why are the changes needed?
This is needed for DSv2 sources to supply stats for cost-based rules in the optimizer.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
This updates the implementation of stats from `DataSourceV2Relation` so tests will fail if stats are accessed before early pushdown for v2 relations.
Closes#25955 from rdblue/move-v2-pushdown.
Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Ryan Blue <blue@apache.org>
### What changes were proposed in this pull request?
To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.
Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the sparkR version number check logic to allow jvm version like `3.0.0-preview`
**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**
We shall revert the changes after 3.0.0-preview release passed.
### Why are the changes needed?
To make the maven release repository to accept the built jars.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
N/A
### What changes were proposed in this pull request?
MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY
### Why are the changes needed?
fix bug
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
add ut
Closes#26321 from yaooqinn/SPARK-29653.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This patch fixes the issue that external listeners are not initialized properly when `spark.sql.hive.metastore.jars` is set to either "maven" or custom list of jar.
("builtin" is not a case here - all jars in Spark classloader are also available in separate classloader)
The culprit is lazy initialization (lazy val or passing builder function) & thread context classloader. HiveClient leverages IsolatedClientLoader to properly load Hive and relevant libraries without issue - to not mess up with Spark classpath it uses separate classloader with leveraging thread context classloader.
But there's a messed-up case - SessionState is being initialized while HiveClient changed the thread context classloader from Spark classloader to Hive isolated one, and streaming query listeners are loaded from changed classloader while initializing SessionState.
This patch forces initializing SessionState in SparkSQLEnv to avoid such case.
### Why are the changes needed?
ClassNotFoundException could occur in spark-sql with specific configuration, as explained above.
### Does this PR introduce any user-facing change?
No, as I don't think end users assume the classloader of external listeners is only containing jars for Hive client.
### How was this patch tested?
New UT added which fails on master branch and passes with the patch.
The error message with master branch when running UT:
```
java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':;
org.apache.spark.sql.AnalysisException: java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':;
at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109)
at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:221)
at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:147)
at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:137)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:59)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.$anonfun$new$2(SparkSQLEnvSuite.scala:44)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.withSystemProperties(SparkSQLEnvSuite.scala:61)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.$anonfun$new$1(SparkSQLEnvSuite.scala:43)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
at org.scalatest.Transformer.apply(Transformer.scala:22)
at org.scalatest.Transformer.apply(Transformer.scala:20)
at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:149)
at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
at org.scalatest.SuperEngine.runTestImpl(Engine.scala:286)
at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:56)
at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:56)
at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:393)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:381)
at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:376)
at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:458)
at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
at org.scalatest.Suite.run(Suite.scala:1124)
at org.scalatest.Suite.run$(Suite.scala:1106)
at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
at org.scalatest.SuperEngine.runImpl(Engine.scala:518)
at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:56)
at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:56)
at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1349)
at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1343)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1343)
at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:1033)
at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:1011)
at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1509)
at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1011)
at org.scalatest.tools.Runner$.run(Runner.scala:850)
at org.scalatest.tools.Runner.run(Runner.scala)
at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:133)
at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:27)
Caused by: java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':
at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1054)
at org.apache.spark.sql.SparkSession.$anonfun$sessionState$2(SparkSession.scala:156)
at scala.Option.getOrElse(Option.scala:189)
at org.apache.spark.sql.SparkSession.sessionState$lzycompute(SparkSession.scala:154)
at org.apache.spark.sql.SparkSession.sessionState(SparkSession.scala:151)
at org.apache.spark.sql.SparkSession.$anonfun$new$3(SparkSession.scala:105)
at scala.Option.map(Option.scala:230)
at org.apache.spark.sql.SparkSession.$anonfun$new$1(SparkSession.scala:105)
at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:164)
at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:183)
at org.apache.spark.sql.hive.client.HiveClientImpl.<init>(HiveClientImpl.scala:127)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:300)
at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:421)
at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:314)
at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:68)
at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:67)
at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$databaseExists$1(HiveExternalCatalog.scala:221)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
... 58 more
Caused by: java.lang.ClassNotFoundException: test.custom.listener.DummyQueryExecutionListener
at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
at java.lang.Class.forName0(Native Method)
at java.lang.Class.forName(Class.java:348)
at org.apache.spark.util.Utils$.classForName(Utils.scala:206)
at org.apache.spark.util.Utils$.$anonfun$loadExtensions$1(Utils.scala:2746)
at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245)
at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245)
at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242)
at scala.collection.AbstractTraversable.flatMap(Traversable.scala:108)
at org.apache.spark.util.Utils$.loadExtensions(Utils.scala:2744)
at org.apache.spark.sql.util.ExecutionListenerManager.$anonfun$new$1(QueryExecutionListener.scala:83)
at org.apache.spark.sql.util.ExecutionListenerManager.$anonfun$new$1$adapted(QueryExecutionListener.scala:82)
at scala.Option.foreach(Option.scala:407)
at org.apache.spark.sql.util.ExecutionListenerManager.<init>(QueryExecutionListener.scala:82)
at org.apache.spark.sql.internal.BaseSessionStateBuilder.$anonfun$listenerManager$2(BaseSessionStateBuilder.scala:293)
at scala.Option.getOrElse(Option.scala:189)
at org.apache.spark.sql.internal.BaseSessionStateBuilder.listenerManager(BaseSessionStateBuilder.scala:293)
at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:320)
at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1051)
... 80 more
```
Closes#26258 from HeartSaVioR/SPARK-29604.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
```
postgres=# select date '2001-09-28' + integer '7';
?column?
------------
2001-10-05
(1 row)postgres=# select integer '7';
int4
------
7
(1 row)
```
Add support for typed integer literal expression from postgreSQL.
### Why are the changes needed?
SPARK-27764 Feature Parity between PostgreSQL and Spark
### Does this PR introduce any user-facing change?
support typed integer lit in SQL
### How was this patch tested?
add uts
Closes#26291 from yaooqinn/SPARK-29629.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Now, `RepartitionByExpression` is allowed at Dataset method `Dataset.repartition()`. But in spark sql, we do not have an equivalent functionality.
In hive, we can use `distribute by`, so it's worth to add a hint to support such function.
Similar jira [SPARK-24940](https://issues.apache.org/jira/browse/SPARK-24940)
## Why are the changes needed?
Make repartition hints consistent with repartition api .
## Does this PR introduce any user-facing change?
This pr intends to support quries below;
```
// SQL cases
- sql("SELECT /*+ REPARTITION(c) */ * FROM t")
- sql("SELECT /*+ REPARTITION(1, c) */ * FROM t")
- sql("SELECT /*+ REPARTITION_BY_RANGE(c) */ * FROM t")
- sql("SELECT /*+ REPARTITION_BY_RANGE(1, c) */ * FROM t")
```
## How was this patch tested?
UT
Closes#25464 from ulysses-you/SPARK-28746.
Lead-authored-by: ulysses <youxiduo@weidian.com>
Co-authored-by: ulysses <646303253@qq.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Add AlterTableRecoverPartitionsStatement and make ALTER TABLE ... RECOVER PARTITIONS go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ALTER TABLE t RECOVER PARTITIONS // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
Yes. When running ALTER TABLE ... RECOVER PARTITIONS Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26269 from huaxingao/spark-29612.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.
Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the PySpark version from `3.0.0.dev0` to `3.0.0`
**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**
We shall revert the changes after 3.0.0-preview release passed.
### Why are the changes needed?
To make the maven release repository to accept the built jars.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
N/A
Closes#26243 from jiangxb1987/3.0.0-preview-prepare.
Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
### What changes were proposed in this pull request?
This PR adds `DROP NAMESPACE` support for V2 catalogs.
### Why are the changes needed?
Currently, you cannot drop namespaces for v2 catalogs.
### Does this PR introduce any user-facing change?
The user can now perform the following:
```SQL
CREATE NAMESPACE mycatalog.ns
DROP NAMESPACE mycatalog.ns
SHOW NAMESPACES IN mycatalog # Will show no namespaces
```
to drop a namespace `ns` inside `mycatalog` V2 catalog.
### How was this patch tested?
Added unit tests.
Closes#26262 from imback82/drop_namespace.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Add LoadDataStatement and make LOAD DATA INTO TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
LOAD DATA INPATH 'filepath' INTO TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running LOAD DATA INTO TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26178 from viirya/SPARK-29521.
Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
When make the `LocalShuffleReaderExec` to leaf node, there exists a potential issue: the leaf node will hide the running query stage and make the unfinished query stage as finished query stage when creating its parent query stage.
This PR make the leaf node to unary node.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Existing tests
Closes#26250 from JkSelf/updateLeafNodeofLocalReaderToUnaryExecNode.
Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Reset the `WritableColumnVector` when getting "next" ColumnarBatch in `RowToColumnarExec`
### Why are the changes needed?
When converting `Iterator[InternalRow]` to `Iterator[ColumnarBatch]`, the vectors used to create a new `ColumnarBatch` should be reset in the iterator's "next()" method.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
N/A
Closes#26137 from rongma1997/reset-WritableColumnVector.
Authored-by: rongma1997 <rong.ma@intel.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
```
hive> select version();
OK
3.1.1 rf4e0529634b6231a0072295da48af466cf2f10b7
Time taken: 2.113 seconds, Fetched: 1 row(s)
```
### Why are the changes needed?
From hive behavior and I guess it is useful for debugging and developing etc.
### Does this PR introduce any user-facing change?
add a misc func
### How was this patch tested?
add ut
Closes#26209 from yaooqinn/SPARK-29554.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Add ShowCreateTableStatement and make SHOW CREATE TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW CREATE TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running SHOW CREATE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26184 from viirya/SPARK-29527.
Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This patch adds the functionality to measure records being written for JDBC writer. In reality, the value is meant to be a number of records being updated from queries, as per JDBC spec it will return updated count.
### Why are the changes needed?
Output metrics for JDBC writer are missing now. The value of "bytesWritten" is also missing, but we can't measure it from JDBC API.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Unit test added.
Closes#26109 from HeartSaVioR/SPARK-29461.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
First, a bit of background on the code being changed. The current code tracks
metric updates for each task, recording which metrics the task is monitoring
and the last update value.
Once a SQL execution finishes, then the metrics for all the stages are
aggregated, by building a list with all (metric ID, value) pairs collected
for all tasks in the stages related to the execution, then grouping by metric
ID, and then calculating the values shown in the UI.
That is full of inefficiencies:
- in normal operation, all tasks will be tracking and updating the same
metrics. So recording the metric IDs per task is wasteful.
- tracking by task means we might be double-counting values if you have
speculative tasks (as a comment in the code mentions).
- creating a list of (metric ID, value) is extremely inefficient, because now
you have a huge map in memory storing boxed versions of the metric IDs and
values.
- same thing for the aggregation part, where now a Seq is built with the values
for each metric ID.
The end result is that for large queries, this code can become both really
slow, thus affecting the processing of events, and memory hungry.
The updated code changes the approach to the following:
- stages track metrics by their ID; this means the stage tracking code
naturally groups values, making aggregation later simpler.
- each metric ID being tracked uses a long array matching the number of
partitions of the stage; this means that it's cheap to update the value of
the metric once a task ends.
- when aggregating, custom code just concatenates the arrays corresponding to
the matching metric IDs; this is cheaper than the previous, boxing-heavy
approach.
The end result is that the listener uses about half as much memory as before
for tracking metrics, since it doesn't need to track metric IDs per task.
I captured heap dumps with the old and the new code during metric aggregation
in the listener, for an execution with 3 stages, 100k tasks per stage, 50
metrics updated per task. The dumps contained just reachable memory - so data
kept by the listener plus the variables in the aggregateMetrics() method.
With the old code, the thread doing aggregation references >1G of memory - and
that does not include temporary data created by the "groupBy" transformation
(for which the intermediate state is not referenced in the aggregation method).
The same thread with the new code references ~250M of memory. The old code uses
about ~250M to track all the metric values for that execution, while the new
code uses about ~130M. (Note the per-thread numbers include the amount used to
track the metrics - so, e.g., in the old case, aggregation was referencing
about ~750M of temporary data.)
I'm also including a small benchmark (based on the Benchmark class) so that we
can measure how much changes to this code affect performance. The benchmark
contains some extra code to measure things the normal Benchmark class does not,
given that the code under test does not really map that well to the
expectations of that class.
Running with the old code (I removed results that don't make much
sense for this benchmark):
```
[info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Linux 4.15.0-66-generic
[info] Intel(R) Core(TM) i7-6820HQ CPU 2.70GHz
[info] metrics aggregation (50 metrics, 100k tasks per stage): Best Time(ms) Avg Time(ms)
[info] --------------------------------------------------------------------------------------
[info] 1 stage(s) 2113 2118
[info] 2 stage(s) 4172 4392
[info] 3 stage(s) 7755 8460
[info]
[info] Stage Count Stage Proc. Time Aggreg. Time
[info] 1 614 1187
[info] 2 620 2480
[info] 3 718 5069
```
With the new code:
```
[info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Linux 4.15.0-66-generic
[info] Intel(R) Core(TM) i7-6820HQ CPU 2.70GHz
[info] metrics aggregation (50 metrics, 100k tasks per stage): Best Time(ms) Avg Time(ms)
[info] --------------------------------------------------------------------------------------
[info] 1 stage(s) 727 886
[info] 2 stage(s) 1722 1983
[info] 3 stage(s) 2752 3013
[info]
[info] Stage Count Stage Proc. Time Aggreg. Time
[info] 1 408 177
[info] 2 389 423
[info] 3 372 660
```
So the new code is faster than the old when processing task events, and about
an order of maginute faster when aggregating metrics.
Note this still leaves room for improvement; for example, using the above
measurements, 600ms is still a huge amount of time to spend in an event
handler. But I'll leave further enhancements for a separate change.
Tested with benchmarking code + existing unit tests.
Closes#26218 from vanzin/SPARK-29562.
Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Track timing info for each rule in optimization phase using `QueryPlanningTracker` in Structured Streaming
### Why are the changes needed?
In Structured Streaming we only track rule info in analysis phase, not in optimization phase.
### Does this PR introduce any user-facing change?
No
Closes#25914 from wenxuanguan/spark-29227.
Authored-by: wenxuanguan <choose_home@126.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Add UncacheTableStatement and make UNCACHE TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
UNCACHE TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running UNCACHE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
New unit tests
Closes#26237 from imback82/uncache_table.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Remove the requirement of fetch_size>=0 from JDBCOptions to allow negative fetch size.
### Why are the changes needed?
Namely, to allow data fetch in stream manner (row-by-row fetch) against MySQL database.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Unit test (JDBCSuite)
This closes#26230 .
Closes#26244 from fuwhu/SPARK-21287-FIX.
Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
# What changes were proposed in this pull request?
Add description for ignoreNullFields, which is commited in #26098 , in DataFrameWriter and readwriter.py.
Enable user to use ignoreNullFields in pyspark.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
run unit tests
Closes#26227 from stczwd/json-generator-doc.
Authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Only use antlr4 to parse the interval string, and remove the duplicated parsing logic from `CalendarInterval`.
### Why are the changes needed?
Simplify the code and fix inconsistent behaviors.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Pass the Jenkins with the updated test cases.
Closes#26190 from cloud-fan/parser.
Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Support SparkSQL use iN/EXISTS with subquery in JOIN condition.
### Why are the changes needed?
Support SQL use iN/EXISTS with subquery in JOIN condition.
### Does this PR introduce any user-facing change?
This PR is for enable user use subquery in `JOIN`'s ON condition. such as we have create three table
```
CREATE TABLE A(id String);
CREATE TABLE B(id String);
CREATE TABLE C(id String);
```
we can do query like :
```
SELECT A.id from A JOIN B ON A.id = B.id and A.id IN (select C.id from C)
```
### How was this patch tested?
ADDED UT
Closes#25854 from AngersZhuuuu/SPARK-29145.
Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Reimplement the iterator in UnsafeExternalRowSorter in database style. This can be done by reusing the `RowIterator` in our code base.
### Why are the changes needed?
During the job in #26164, after involving a var `isReleased` in `hasNext`, there's possible that `isReleased` is false when calling `hasNext`, but it becomes true before calling `next`. A safer way is using database-style iterator: `advanceNext` and `getRow`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing UT.
Closes#26229 from xuanyuanking/SPARK-21492-follow-up.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add CacheTableStatement and make CACHE TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
CACHE TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running CACHE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26179 from viirya/SPARK-29522.
Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This is a follow-up of #24052 to correct assert condition.
### Why are the changes needed?
To test IllegalArgumentException condition..
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Manual Test (during fixing of SPARK-29453 find this issue)
Closes#26234 from 07ARB/SPARK-29571.
Authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This is a follow-up of https://github.com/apache/spark/pull/26189 to regenerate the result on EC2.
### Why are the changes needed?
This will be used for the other PR reviews.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
N/A.
Closes#26233 from dongjoon-hyun/SPARK-29533.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
### What changes were proposed in this pull request?
`OptimizeLocalShuffleReader` rule is very conservative and gives up optimization as long as there are extra shuffles introduced. It's very likely that most of the added local shuffle readers are fine and only one introduces extra shuffle.
However, it's very hard to make `OptimizeLocalShuffleReader` optimal, a simple workaround is to run this rule again right before executing a query stage.
### Why are the changes needed?
Optimize more shuffle reader to local shuffle reader.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
existing ut
Closes#26207 from JkSelf/resolve-multi-joins-issue.
Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add RefreshTableStatement and make REFRESH TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
REFRESH TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running REFRESH TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
New unit tests
Closes#26183 from imback82/refresh_table.
Lead-authored-by: Terry Kim <yuminkim@gmail.com>
Co-authored-by: Terry Kim <terryk@terrys-mbp-2.lan>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
### What changes were proposed in this pull request?
This moves the tracking of active queries from a per SparkSession state, to the shared SparkSession for better safety in isolated Spark Session environments.
### Why are the changes needed?
We have checks to prevent the restarting of the same stream on the same spark session, but we can actually make that better in multi-tenant environments by actually putting that state in the SharedState instead of SessionState. This would allow a more comprehensive check for multi-tenant clusters.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Added tests to StreamingQueryManagerSuite
Closes#26018 from brkyvz/sharedStreamingQueryManager.
Lead-authored-by: Burak Yavuz <burak@databricks.com>
Co-authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
### What changes were proposed in this pull request?
This PR adds `CREATE NAMESPACE` support for V2 catalogs.
### Why are the changes needed?
Currently, you cannot explicitly create namespaces for v2 catalogs.
### Does this PR introduce any user-facing change?
The user can now perform the following:
```SQL
CREATE NAMESPACE mycatalog.ns
```
to create a namespace `ns` inside `mycatalog` V2 catalog.
### How was this patch tested?
Added unit tests.
Closes#26166 from imback82/create_namespace.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add ShowPartitionsStatement and make SHOW PARTITIONS go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users.
### Does this PR introduce any user-facing change?
Yes. When running SHOW PARTITIONS, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26198 from huaxingao/spark-29539.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
### What changes were proposed in this pull request?
Add TruncateTableStatement and make TRUNCATE TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
TRUNCATE TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running TRUNCATE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
Unit tests.
Closes#26174 from viirya/SPARK-29517.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
We shall have a new mechanism that the downstream operators may notify its parents that they may release the output data stream. In this PR, we implement the mechanism as below:
- Add function named `cleanupResources` in SparkPlan, which default call children's `cleanupResources` function, the operator which need a resource cleanup should rewrite this with the self cleanup and also call `super.cleanupResources`, like SortExec in this PR.
- Add logic support on the trigger side, in this PR is SortMergeJoinExec, which make sure and call the `cleanupResources` to do the cleanup job for all its upstream(children) operator.
### Why are the changes needed?
Bugfix for SortMergeJoin memory leak, and implement a general framework for SparkPlan resource cleanup.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
UT: Add new test suite JoinWithResourceCleanSuite to check both standard and code generation scenario.
Integrate Test: Test with driver/executor default memory set 1g, local mode 10 thread. The below test(thanks taosaildrone for providing this test [here](https://github.com/apache/spark/pull/23762#issuecomment-463303175)) will pass with this PR.
```
from pyspark.sql.functions import rand, col
spark.conf.set("spark.sql.join.preferSortMergeJoin", "true")
spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1)
# spark.conf.set("spark.sql.sortMergeJoinExec.eagerCleanupResources", "true")
r1 = spark.range(1, 1001).select(col("id").alias("timestamp1"))
r1 = r1.withColumn('value', rand())
r2 = spark.range(1000, 1001).select(col("id").alias("timestamp2"))
r2 = r2.withColumn('value2', rand())
joined = r1.join(r2, r1.timestamp1 == r2.timestamp2, "inner")
joined = joined.coalesce(1)
joined.explain()
joined.show()
```
Closes#26164 from xuanyuanking/SPARK-21492.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
As I have comment in [SPARK-29516](https://github.com/apache/spark/pull/26172#issuecomment-544364977)
SparkSession.sql() method parse process not under current sparksession's conf, so some configuration about parser is not valid in multi-thread situation.
In this pr, we add a SQLConf parameter to AbstractSqlParser and initial it with SessionState's conf.
Then for each SparkSession's parser process. It will use's it's own SessionState's SQLConf and to be thread safe
### Why are the changes needed?
Fix bug
### Does this PR introduce any user-facing change?
NO
### How was this patch tested?
NO
Closes#26187 from AngersZhuuuu/SPARK-29530.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Only invoke `checkAndGlobPathIfNecessary()` when we have to use `InMemoryFileIndex`.
### Why are the changes needed?
Avoid unnecessary function invocation.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Pass Jenkins.
Closes#26196 from Ngone51/dev-avoid-unnecessary-invocation-on-globpath.
Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
I extended `ExtractBenchmark` to support the `INTERVAL` type of the `source` parameter of the `date_part` function.
### Why are the changes needed?
- To detect performance issues while changing implementation of the `date_part` function in the future.
- To find out current performance bottlenecks in `date_part` for the `INTERVAL` type
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By running the benchmark and print out produced values per each `field` value.
Closes#26175 from MaxGekk/extract-interval-benchmark.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Added new benchmark `IntervalBenchmark` to measure performance of interval related functions. In the PR, I added benchmarks for casting strings to interval. In particular, interval strings with `interval` prefix and without it because there is special code for this da576a737c/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java (L100-L103) . And also I added benchmarks for different number of units in interval strings, for example 1 unit is `interval 10 years`, 2 units w/o interval is `10 years 5 months`, and etc.
### Why are the changes needed?
- To find out current performance issues in casting to intervals
- The benchmark can be used while refactoring/re-implementing `CalendarInterval.fromString()` or `CalendarInterval.fromCaseInsensitiveString()`.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By running the benchmark via the command:
```shell
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.IntervalBenchmark"
```
Closes#26189 from MaxGekk/interval-from-string-benchmark.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
```
bit_and(expression) -- The bitwise AND of all non-null input values, or null if none
bit_or(expression) -- The bitwise OR of all non-null input values, or null if none
```
More details:
https://www.postgresql.org/docs/9.3/functions-aggregate.html
### Why are the changes needed?
Postgres, Mysql and many other popular db support them.
### Does this PR introduce any user-facing change?
add two bit agg
### How was this patch tested?
add ut
Closes#26155 from yaooqinn/SPARK-27879.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR fix Fix the associated location already exists in `SQLQueryTestSuite`:
```
build/sbt "~sql/test-only *SQLQueryTestSuite -- -z postgreSQL/join.sql"
...
[info] - postgreSQL/join.sql *** FAILED *** (35 seconds, 420 milliseconds)
[info] postgreSQL/join.sql
[info] Expected "[]", but got "[org.apache.spark.sql.AnalysisException
[info] Can not create the managed table('`default`.`tt3`'). The associated location('file:/root/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/tt3') already exists.;]" Result did not match for query #108
```
### Why are the changes needed?
Fix bug.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
N/A
Closes#26181 from wangyum/TestError.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Add RepairTableStatement and make REPAIR TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
MSCK REPAIR TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running MSCK REPAIR TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
New unit tests
Closes#26168 from imback82/repair_table.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
### What changes were proposed in this pull request?
Current Spark SQL `SHOW FUNCTIONS` don't show `!=`, `<>`, `between`, `case`
But these expressions is truly functions. We should show it in SQL `SHOW FUNCTIONS`
### Why are the changes needed?
SHOW FUNCTIONS show '!=', '<>' , 'between', 'case'
### Does this PR introduce any user-facing change?
SHOW FUNCTIONS show '!=', '<>' , 'between', 'case'
### How was this patch tested?
UT
Closes#26053 from AngersZhuuuu/SPARK-29379.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The `date_part()` function can accept the `source` parameter of the `INTERVAL` type (`CalendarIntervalType`). The following values of the `field` parameter are supported:
- `"MILLENNIUM"` (`"MILLENNIA"`, `"MIL"`, `"MILS"`) - number of millenniums in the given interval. It is `YEAR / 1000`.
- `"CENTURY"` (`"CENTURIES"`, `"C"`, `"CENT"`) - number of centuries in the interval calculated as `YEAR / 100`.
- `"DECADE"` (`"DECADES"`, `"DEC"`, `"DECS"`) - decades in the `YEAR` part of the interval calculated as `YEAR / 10`.
- `"YEAR"` (`"Y"`, `"YEARS"`, `"YR"`, `"YRS"`) - years in a values of `CalendarIntervalType`. It is `MONTHS / 12`.
- `"QUARTER"` (`"QTR"`) - a quarter of year calculated as `MONTHS / 3 + 1`
- `"MONTH"` (`"MON"`, `"MONS"`, `"MONTHS"`) - the months part of the interval calculated as `CalendarInterval.months % 12`
- `"DAY"` (`"D"`, `"DAYS"`) - total number of days in `CalendarInterval.microseconds`
- `"HOUR"` (`"H"`, `"HOURS"`, `"HR"`, `"HRS"`) - the hour part of the interval.
- `"MINUTE"` (`"M"`, `"MIN"`, `"MINS"`, `"MINUTES"`) - the minute part of the interval.
- `"SECOND"` (`"S"`, `"SEC"`, `"SECONDS"`, `"SECS"`) - the seconds part with fractional microsecond part.
- `"MILLISECONDS"` (`"MSEC"`, `"MSECS"`, `"MILLISECON"`, `"MSECONDS"`, `"MS"`) - the millisecond part of the interval with fractional microsecond part.
- `"MICROSECONDS"` (`"USEC"`, `"USECS"`, `"USECONDS"`, `"MICROSECON"`, `"US"`) - the total number of microseconds in the `second`, `millisecond` and `microsecond` parts of the given interval.
- `"EPOCH"` - the total number of seconds in the interval including the fractional part with microsecond precision. Here we assume 365.25 days per year (leap year every four years).
For example:
```sql
> SELECT date_part('days', interval 1 year 10 months 5 days);
5
> SELECT date_part('seconds', interval 30 seconds 1 milliseconds 1 microseconds);
30.001001
```
### Why are the changes needed?
To maintain feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT)
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
- Added new test suite `IntervalExpressionsSuite`
- Add new test cases to `date_part.sql`
Closes#25981 from MaxGekk/extract-from-intervals.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
A followup of [#25295](https://github.com/apache/spark/pull/25295).
1) change the logWarning to logDebug in `OptimizeLocalShuffleReader`.
2) update the test to check whether query stage reuse can work well with local shuffle reader.
### Why are the changes needed?
make code robust
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
existing tests
Closes#26157 from JkSelf/followup-25295.
Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The handling of the catalog across plans should be as follows ([SPARK-29014](https://issues.apache.org/jira/browse/SPARK-29014)):
* The *current* catalog should be used when no catalog is specified
* The default catalog is the catalog *current* is initialized to
* If the *default* catalog is not set, then *current* catalog is the built-in Spark session catalog.
This PR addresses the issue where *current* catalog usage is not followed as describe above.
### Why are the changes needed?
It is a bug as described in the previous section.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Unit tests added.
Closes#26120 from imback82/cleanup_catalog.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add `AnalyzeTableStatement` and `AnalyzeColumnStatement`, and make ANALYZE TABLE go through the same catalog/table resolution framework of v2 commands.
### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g.
```
USE my_catalog
DESC t // success and describe the table t from my_catalog
ANALYZE TABLE t // report table not found as there is no table t in the session catalog
```
### Does this PR introduce any user-facing change?
yes. When running ANALYZE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.
### How was this patch tested?
new tests
Closes#26129 from cloud-fan/analyze-table.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
This PR proposes a few typos:
1. Sparks => Spark's
2. parallize => parallelize
3. doesnt => doesn't
Closes#26140 from plusplusjiajia/fix-typos.
Authored-by: Jiajia Li <jiajia.li@intel.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
BIT_COUNT(N) - Returns the number of bits that are set in the argument N as an unsigned 64-bit integer, or NULL if the argument is NULL
### Why are the changes needed?
Supported by MySQL,Microsoft SQL Server ,etc.
### Does this PR introduce any user-facing change?
add a built-in function
### How was this patch tested?
add uts
Closes#26139 from yaooqinn/SPARK-29491.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This PR takes over #19788. After we split the shuffle fetch protocol from `OpenBlock` in #24565, this optimization can be extended in the new shuffle protocol. Credit to yucai, closes#19788.
### What changes were proposed in this pull request?
This PR adds the support for continuous shuffle block fetching in batch:
- Shuffle client changes:
- Add new feature tag `spark.shuffle.fetchContinuousBlocksInBatch`, implement the decision logic in `BlockStoreShuffleReader`.
- Merge the continuous shuffle block ids in batch if needed in ShuffleBlockFetcherIterator.
- Shuffle server changes:
- Add support in `ExternalBlockHandler` for the external shuffle service side.
- Make `ShuffleBlockResolver.getBlockData` accept getting block data by range.
- Protocol changes:
- Add new block id type `ShuffleBlockBatchId` represent continuous shuffle block ids.
- Extend `FetchShuffleBlocks` and `OneForOneBlockFetcher`.
- After the new shuffle fetch protocol completed in #24565, the backward compatibility for external shuffle service can be controlled by `spark.shuffle.useOldFetchProtocol`.
### Why are the changes needed?
In adaptive execution, one reducer may fetch multiple continuous shuffle blocks from one map output file. However, as the original approach, each reducer needs to fetch those 10 reducer blocks one by one. This way needs many IO and impacts performance. This PR is to support fetching those continuous shuffle blocks in one IO (batch way). See below example:
The shuffle block is stored like below:
![image](https://user-images.githubusercontent.com/2989575/51654634-c37fbd80-1fd3-11e9-935e-5652863676c3.png)
The ShuffleId format is s"shuffle_$shuffleId_$mapId_$reduceId", referring to BlockId.scala.
In adaptive execution, one reducer may want to read output for reducer 5 to 14, whose block Ids are from shuffle_0_x_5 to shuffle_0_x_14.
Before this PR, Spark needs 10 disk IOs + 10 network IOs for each output file.
After this PR, Spark only needs 1 disk IO and 1 network IO. This way can reduce IO dramatically.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Add new UT.
Integrate test with setting `spark.sql.adaptive.enabled=true`.
Closes#26040 from xuanyuanking/SPARK-9853.
Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
bool_or(x) <=> any/some(x) <=> max(x)
bool_and(x) <=> every(x) <=> min(x)
Args:
x: boolean
### Why are the changes needed?
PostgreSQL, Presto and Vertica, etc also support this feature:
### Does this PR introduce any user-facing change?
add new functions support
### How was this patch tested?
add ut
Closes#26126 from yaooqinn/SPARK-27880.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Proposed new expression `SubtractDates` which is used in `date1` - `date2`. It has the `INTERVAL` type, and returns the interval from `date1` (inclusive) and `date2` (exclusive). For example:
```sql
> select date'tomorrow' - date'yesterday';
interval 2 days
```
Closes#26034
### Why are the changes needed?
- To conform the SQL standard which states the result type of `date operand 1` - `date operand 2` must be the interval type. See [4.5.3 Operations involving datetimes and intervals](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt).
- Improve Spark SQL UX and allow mixing date and timestamp in subtractions. For example: `select timestamp'now' + (date'2019-10-01' - date'2019-09-15')`
### Does this PR introduce any user-facing change?
Before the query below returns number of days:
```sql
spark-sql> select date'2019-10-05' - date'2018-09-01';
399
```
After it returns an interval:
```sql
spark-sql> select date'2019-10-05' - date'2018-09-01';
interval 1 years 1 months 4 days
```
### How was this patch tested?
- by new tests in `DateExpressionsSuite` and `TypeCoercionSuite`.
- by existing tests in `date.sql`
Closes#26112 from MaxGekk/date-subtract.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
### What changes were proposed in this pull request?
A followup of https://github.com/apache/spark/pull/25295
This PR proposes a few code cleanups:
1. rename the special `getMapSizesByExecutorId` to `getMapSizesByMapIndex`
2. rename the parameter `mapId` to `mapIndex` as that's really a mapper index.
3. `BlockStoreShuffleReader` should take `blocksByAddress` directly instead of a map id.
4. rename `getMapReader` to `getReaderForOneMapper` to be more clearer.
### Why are the changes needed?
make code easier to understand
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26128 from cloud-fan/followup.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
Updating univocity-parsers version to 2.8.3, which adds support for multiple character delimiters
Moving univocity-parsers version to spark-parent pom dependencyManagement section
Adding new utility method to build multi-char delimiter string, which delegates to existing one
Adding tests for multiple character delimited CSV
### What changes were proposed in this pull request?
Adds support for parsing CSV data using multiple-character delimiters. Existing logic for converting the input delimiter string to characters was kept and invoked in a loop. Project dependencies were updated to remove redundant declaration of `univocity-parsers` version, and also to change that version to the latest.
### Why are the changes needed?
It is quite common for people to have delimited data, where the delimiter is not a single character, but rather a sequence of characters. Currently, it is difficult to handle such data in Spark (typically needs pre-processing).
### Does this PR introduce any user-facing change?
Yes. Specifying the "delimiter" option for the DataFrame read, and providing more than one character, will no longer result in an exception. Instead, it will be converted as before and passed to the underlying library (Univocity), which has accepted multiple character delimiters since 2.8.0.
### How was this patch tested?
The `CSVSuite` tests were confirmed passing (including new methods), and `sbt` tests for `sql` were executed.
Closes#26027 from jeff303/SPARK-24540.
Authored-by: Jeff Evans <jeffrey.wayne.evans@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
When inserting a value into a column with the different data type, Spark performs type coercion. Currently, we support 3 policies for the store assignment rules: ANSI, legacy and strict, which can be set via the option "spark.sql.storeAssignmentPolicy":
1. ANSI: Spark performs the type coercion as per ANSI SQL. In practice, the behavior is mostly the same as PostgreSQL. It disallows certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean`. It will throw a runtime exception if the value is out-of-range(overflow).
2. Legacy: Spark allows the type coercion as long as it is a valid `Cast`, which is very loose. E.g., converting either `string` to `int` or `double` to `boolean` is allowed. It is the current behavior in Spark 2.x for compatibility with Hive. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of Byte type, the result is 1.
3. Strict: Spark doesn't allow any possible precision loss or data truncation in store assignment, e.g., converting either `double` to `int` or `decimal` to `double` is allowed. The rules are originally for Dataset encoder. As far as I know, no mainstream DBMS is using this policy by default.
Currently, the V1 data source uses "Legacy" policy by default, while V2 uses "Strict". This proposal is to use "ANSI" policy by default for both V1 and V2 in Spark 3.0.
### Why are the changes needed?
Following the ANSI SQL standard is most reasonable among the 3 policies.
### Does this PR introduce any user-facing change?
Yes.
The default store assignment policy is ANSI for both V1 and V2 data sources.
### How was this patch tested?
Unit test
Closes#26107 from gengliangwang/ansiPolicyAsDefault.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
Implement a rule in the new adaptive execution framework introduced in [SPARK-23128](https://issues.apache.org/jira/browse/SPARK-23128). This rule is used to optimize the shuffle reader to local shuffle reader when smj is converted to bhj in adaptive execution.
## How was this patch tested?
Existing tests
Closes#25295 from JkSelf/localShuffleOptimization.
Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
move the statement logical plans that were created for v2 commands to a new file `statements.scala`, under the same package of `v2Commands.scala`.
This PR also includes some minor cleanups:
1. remove `private[sql]` from `ParsedStatement` as it's in the private package.
2. remove unnecessary override of `output` and `children`.
3. add missing classdoc.
### Why are the changes needed?
Similar to https://github.com/apache/spark/pull/26111 , this is to better organize the logical plans of data source v2.
It's a bit weird to put the statements in the package `org.apache.spark.sql.catalyst.plans.logical.sql` as `sql` is not a good sub-package name in Spark SQL.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
existing tests
Closes#26125 from cloud-fan/statement.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
There will be 2 times unsafeProjection convert operation When we read a Parquet data file use non-vectorized mode:
1. `ParquetGroupConverter` call unsafeProjection function to covert `SpecificInternalRow` to `UnsafeRow` every times when read Parquet data file use `ParquetRecordReader`.
2. `ParquetFileFormat` will call unsafeProjection function to covert this `UnsafeRow` to another `UnsafeRow` again when partitionSchema is not empty in DataSourceV1 branch, and `PartitionReaderWithPartitionValues` will always do this convert operation in DataSourceV2 branch.
In this pr, remove `unsafeProjection` convert operation in `ParquetGroupConverter` and change `ParquetRecordReader` to produce `SpecificInternalRow` instead of `UnsafeRow`.
### Why are the changes needed?
The first time convert in `ParquetGroupConverter` is redundant and `ParquetRecordReader` return a `InternalRow(SpecificInternalRow)` is enough.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Unit Test
Closes#26106 from LuciferYang/spark-parquet-unsafe-projection.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Refine the document of v2 session catalog config, to clearly explain what it is, when it should be used and how to implement it.
### Why are the changes needed?
Make this config more understandable
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Pass the Jenkins with the newly updated test cases.
Closes#26071 from cloud-fan/config.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to fix the behavior of `mode("default")` to set `SaveMode.ErrorIfExists`. Also, this PR updates the exception message by adding `default` explicitly.
### Why are the changes needed?
This is reported during `GRAPH API` PR. This builder pattern should work like the documentation.
### Does this PR introduce any user-facing change?
Yes if the app has multiple `mode()` invocation including `mode("default")` and the `mode("default")` is the last invocation. This is really a corner case.
- Previously, the last invocation was handled as `No-Op`.
- After this bug fix, it will work like the documentation.
### How was this patch tested?
Pass the Jenkins with the newly added test case.
Closes#26094 from dongjoon-hyun/SPARK-29442.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
In the PR, I propose to move interval parsing to `CalendarInterval.fromCaseInsensitiveString()` which throws an `IllegalArgumentException` for invalid strings, and reuse it from `CalendarInterval.fromString()`. The former one handles `IllegalArgumentException` only and returns `NULL` for invalid interval strings. This will allow to support interval strings without the `interval` prefix in casting strings to intervals and in interval type constructor because they use `fromString()` for parsing string intervals.
For example:
```sql
spark-sql> select cast('1 year 10 days' as interval);
interval 1 years 1 weeks 3 days
spark-sql> SELECT INTERVAL '1 YEAR 10 DAYS';
interval 1 years 1 weeks 3 days
```
### Why are the changes needed?
To maintain feature parity with PostgreSQL which supports interval strings without prefix:
```sql
# select interval '2 months 1 microsecond';
interval
------------------------
2 mons 00:00:00.000001
```
and to improve Spark SQL UX.
### Does this PR introduce any user-facing change?
Yes, previously parsing of interval strings without `interval` gives `NULL`:
```sql
spark-sql> select interval '2 months 1 microsecond';
NULL
```
After:
```sql
spark-sql> select interval '2 months 1 microsecond';
interval 2 months 1 microseconds
```
### How was this patch tested?
- Added new tests to `CalendarIntervalSuite.java`
- A test for casting strings to intervals in `CastSuite`
- Test for interval type constructor from strings in `ExpressionParserSuite`
Closes#26079 from MaxGekk/interval-str-without-prefix.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Currently, `SHOW NAMESPACES` and `SHOW DATABASES` are separate code paths. This PR merges two implementations.
### Why are the changes needed?
To remove code/behavior duplication
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Added new unit tests.
Closes#26006 from imback82/combine_show.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR adds 2 changes regarding exception handling in `SQLQueryTestSuite` and `ThriftServerQueryTestSuite`
- fixes an expected output sorting issue in `ThriftServerQueryTestSuite` as if there is an exception then there is no need for sort
- introduces common exception handling in those 2 suites with a new `handleExceptions` method
### Why are the changes needed?
Currently `ThriftServerQueryTestSuite` passes on master, but it fails on one of my PRs (https://github.com/apache/spark/pull/23531) with this error (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/111651/testReport/org.apache.spark.sql.hive.thriftserver/ThriftServerQueryTestSuite/sql_3/):
```
org.scalatest.exceptions.TestFailedException: Expected "
[Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit
org.apache.spark.SparkException]
", but got "
[org.apache.spark.SparkException
Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit]
" Result did not match for query #4 WITH RECURSIVE r(level) AS ( VALUES (0) UNION ALL SELECT level + 1 FROM r ) SELECT * FROM r
```
The unexpected reversed order of expected output (error message comes first, then the exception class) is due to this line: https://github.com/apache/spark/pull/26028/files#diff-b3ea3021602a88056e52bf83d8782de8L146. It should not sort the expected output if there was an error during execution.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#26028 from peter-toth/SPARK-29359-better-exception-handling.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
### What changes were proposed in this pull request?
Revert this commit 18b7ad2fc5.
### Why are the changes needed?
See https://github.com/apache/spark/pull/16304#discussion_r92753590
### Does this PR introduce any user-facing change?
Yes
### How was this patch tested?
There is no test for that.
Closes#26101 from MaxGekk/revert-mean-seconds-per-month.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
### What changes were proposed in this pull request?
Use `.sameElements` to compare (non-nested) arrays, as `Arrays.deep` is removed in 2.13 and wasn't the best way to do this in the first place.
### Why are the changes needed?
To compile with 2.13.
### Does this PR introduce any user-facing change?
None.
### How was this patch tested?
Existing tests.
Closes#26073 from srowen/SPARK-29416.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Replace `Unit` with equivalent `()` where code refers to the `Unit` companion object.
### Why are the changes needed?
It doesn't compile otherwise in Scala 2.13.
- https://github.com/scala/scala/blob/v2.13.0/src/library/scala/Unit.scala#L30
### Does this PR introduce any user-facing change?
Should be no behavior change at all.
### How was this patch tested?
Existing tests.
Closes#26070 from srowen/SPARK-29411.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR adds an accumulator that computes a global aggregate over a number of rows. A user can define an arbitrary number of aggregate functions which can be computed at the same time.
The accumulator uses the standard technique for implementing (interpreted) aggregation in Spark. It uses projections and manual updates for each of the aggregation steps (initialize buffer, update buffer with new input row, merge two buffers and compute the final result on the buffer). Note that two of the steps (update and merge) use the aggregation buffer both as input and output.
Accumulators do not have an explicit point at which they get serialized. A somewhat surprising side effect is that the buffers of a `TypedImperativeAggregate` go over the wire as-is instead of serializing them. The merging logic for `TypedImperativeAggregate` assumes that the input buffer contains serialized buffers, this is violated by the accumulator's implicit serialization. In order to get around this I have added `mergeBuffersObjects` method that merges two unserialized buffers to `TypedImperativeAggregate`.
### Why are the changes needed?
This is the mechanism we are going to use to implement observable metrics.
### Does this PR introduce any user-facing change?
No, not yet.
### How was this patch tested?
Added `AggregatingAccumulator` test suite.
Closes#26012 from hvanhovell/SPARK-29346.
Authored-by: herman <herman@databricks.com>
Signed-off-by: herman <herman@databricks.com>
### What changes were proposed in this pull request?
DataSourceV2 Exec classes (ShowTablesExec, ShowNamespacesExec, etc.) all extend LeafExecNode. This results in running a job when executeCollect() is called. This breaks the previous behavior [SPARK-19650](https://issues.apache.org/jira/browse/SPARK-19650).
A new command physical operator will be introduced form which all V2 Exec classes derive to avoid running a job.
### Why are the changes needed?
It is a bug since the current behavior runs a spark job, which breaks the existing behavior: [SPARK-19650](https://issues.apache.org/jira/browse/SPARK-19650).
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Existing unit tests.
Closes#26048 from imback82/dsv2_command.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Invocations like `sc.parallelize(Array((1,2)))` cause a compile error in 2.13, like:
```
[ERROR] [Error] /Users/seanowen/Documents/spark_2.13/core/src/test/scala/org/apache/spark/ShuffleSuite.scala:47: overloaded method value apply with alternatives:
(x: Unit,xs: Unit*)Array[Unit] <and>
(x: Double,xs: Double*)Array[Double] <and>
(x: Float,xs: Float*)Array[Float] <and>
(x: Long,xs: Long*)Array[Long] <and>
(x: Int,xs: Int*)Array[Int] <and>
(x: Char,xs: Char*)Array[Char] <and>
(x: Short,xs: Short*)Array[Short] <and>
(x: Byte,xs: Byte*)Array[Byte] <and>
(x: Boolean,xs: Boolean*)Array[Boolean]
cannot be applied to ((Int, Int), (Int, Int), (Int, Int), (Int, Int))
```
Using a `Seq` instead appears to resolve it, and is effectively equivalent.
### Why are the changes needed?
To better cross-build for 2.13.
### Does this PR introduce any user-facing change?
None.
### How was this patch tested?
Existing tests.
Closes#26062 from srowen/SPARK-29401.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Syntax like `'foo` is deprecated in Scala 2.13. Replace usages with `Symbol("foo")`
### Why are the changes needed?
Avoids ~50 deprecation warnings when attempting to build with 2.13.
### Does this PR introduce any user-facing change?
None, should be no functional change at all.
### How was this patch tested?
Existing tests.
Closes#26061 from srowen/SPARK-29392.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Reimplement `org.apache.spark.sql.catalyst.util.QuantileSummaries#merge` and add a test-case showing the previous bug.
### Why are the changes needed?
The original Greenwald-Khanna paper, from which the algorithm behind `approxQuantile` was taken, does not cover how to merge the result of multiple parallel QuantileSummaries. The current implementation violates some invariants and therefore the effective error can be larger than the specified.
### Does this PR introduce any user-facing change?
Yes, for same cases, the results from `approxQuantile` (`percentile_approx` in SQL) will now be within the expected error margin. For example:
```scala
var values = (1 to 100).toArray
val all_quantiles = values.indices.map(i => (i+1).toDouble / values.length).toArray
for (n <- 0 until 5) {
var df = spark.sparkContext.makeRDD(values).toDF("value").repartition(5)
val all_answers = df.stat.approxQuantile("value", all_quantiles, 0.1)
val all_answered_ranks = all_answers.map(ans => values.indexOf(ans)).toArray
val error = all_answered_ranks.zipWithIndex.map({ case (answer, expected) => Math.abs(expected - answer) }).toArray
val max_error = error.max
print(max_error + "\n")
}
```
In the current build it returns:
```
16
12
10
11
17
```
I couldn't run the code with this patch applied to double check the implementation. Can someone please confirm it now outputs at most `10`, please?
### How was this patch tested?
A new unit test was added to uncover the previous bug.
Closes#26029 from sitegui/SPARK-29336.
Authored-by: Guilherme <sitegui@sitegui.com.br>
Signed-off-by: Sean Owen <sean.owen@databricks.com>