### What changes were proposed in this pull request?
This reverts commit 43a73e387c. It sets `INT96` as the timestamp type while saving timestamps to parquet files.
### Why are the changes needed?
To be compatible with Hive and Presto that don't support the `TIMESTAMP_MICROS` type in current stable releases.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By existing test suites.
Closes#28450 from MaxGekk/parquet-int96.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Push the rebase logic to the lower level of the parquet vectorized reader, to make the final code more vectorization-friendly.
### Why are the changes needed?
Parquet vectorized reader is carefully implemented, to make it more likely to be vectorized by the JVM. However, the newly added datetime rebase degrade the performance a lot, as it breaks vectorization, even if the datetime values don't need to rebase (this is very likely as dates before 1582 is rare).
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
Run part of the `DateTimeRebaseBenchmark` locally. The results:
before this patch
```
[info] Load dates from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1582, vec on, rebase off 2677 2838 142 37.4 26.8 1.0X
[info] after 1582, vec on, rebase on 3828 4331 805 26.1 38.3 0.7X
[info] before 1582, vec on, rebase off 2903 2926 34 34.4 29.0 0.9X
[info] before 1582, vec on, rebase on 4163 4197 38 24.0 41.6 0.6X
[info] Load timestamps from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1900, vec on, rebase off 3537 3627 104 28.3 35.4 1.0X
[info] after 1900, vec on, rebase on 6891 7010 105 14.5 68.9 0.5X
[info] before 1900, vec on, rebase off 3692 3770 72 27.1 36.9 1.0X
[info] before 1900, vec on, rebase on 7588 7610 30 13.2 75.9 0.5X
```
After this patch
```
[info] Load dates from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1582, vec on, rebase off 2758 2944 197 36.3 27.6 1.0X
[info] after 1582, vec on, rebase on 2908 2966 51 34.4 29.1 0.9X
[info] before 1582, vec on, rebase off 2840 2878 37 35.2 28.4 1.0X
[info] before 1582, vec on, rebase on 3407 3433 24 29.4 34.1 0.8X
[info] Load timestamps from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] after 1900, vec on, rebase off 3861 4003 139 25.9 38.6 1.0X
[info] after 1900, vec on, rebase on 4194 4283 77 23.8 41.9 0.9X
[info] before 1900, vec on, rebase off 3849 3937 79 26.0 38.5 1.0X
[info] before 1900, vec on, rebase on 7512 7546 55 13.3 75.1 0.5X
```
Date type is 30% faster if the values don't need to rebase, 20% faster if need to rebase.
Timestamp type is 60% faster if the values don't need to rebase, no difference if need to rebase.
Closes#28406 from cloud-fan/perf.
Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
The collect_set() aggregate function should produce a set of distinct elements. When the column argument's type is BinayType this is not the case.
Example:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.expressions.Window
case class R(id: String, value: String, bytes: Array[Byte])
def makeR(id: String, value: String) = R(id, value, value.getBytes)
val df = Seq(makeR("a", "dog"), makeR("a", "cat"), makeR("a", "cat"), makeR("b", "fish")).toDF()
// In the example below "bytesSet" erroneously has duplicates but "stringSet" does not (as expected).
df.agg(collect_set('value) as "stringSet", collect_set('bytes) as "byteSet").show(truncate=false)
// The same problem is displayed when using window functions.
val win = Window.partitionBy('id).rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)
val result = df.select(
collect_set('value).over(win) as "stringSet",
collect_set('bytes).over(win) as "bytesSet"
)
.select('stringSet, 'bytesSet, size('stringSet) as "stringSetSize", size('bytesSet) as "bytesSetSize")
.show()
```
We use a HashSet buffer to accumulate the results, the problem is that arrays equality in Scala don't behave as expected, arrays ara just plain java arrays and the equality don't compare the content of the arrays
Array(1, 2, 3) == Array(1, 2, 3) => False
The result is that duplicates are not removed in the hashset
The solution proposed is that in the last stage, when we have all the data in the Hashset buffer, we delete duplicates changing the type of the elements and then transform it to the original type.
This transformation is only applied when we have a BinaryType
### Why are the changes needed?
Fix the bug explained
### Does this PR introduce any user-facing change?
Yes. Now `collect_set()` correctly deduplicates array of byte.
### How was this patch tested?
Unit testing
Closes#28351 from planga82/feature/SPARK-31500_COLLECT_SET_bug.
Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Add test cases covering all interval units: MICROSECOND MILLISECOND SECOND MINUTE HOUR DAY WEEK MONTH YEAR
### Why are the changes needed?
For test coverage.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Test only.
Closes#28418 from xuanyuanking/SPARK-28424.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
In the PR, I propose to fix two legacy timestamp formatter `LegacySimpleTimestampFormatter` and `LegacyFastTimestampFormatter` to perform micros rebasing in parsing/formatting from/to strings.
### Why are the changes needed?
Legacy timestamps formatters operate on the hybrid calendar (Julian + Gregorian), so, the input micros should be rebased to have the same date-time fields as in Proleptic Gregorian calendar used by Spark SQL, see SPARK-26651.
### Does this PR introduce any user-facing change?
Yes
### How was this patch tested?
Added tests to `TimestampFormatterSuite`
Closes#28408 from MaxGekk/fix-rebasing-in-legacy-timestamp-formatter.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In `CTESubstitution`, resolve CTE relations first, then traverse the main plan only once to substitute CTE relations.
### Why are the changes needed?
Currently we will traverse the main query many times (if there are many CTE relations), which can be pretty slow if the main query is large.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
local perf test
```
scala> :pa
// Entering paste mode (ctrl-D to finish)
def test(i: Int): Unit = 1.to(i).foreach { _ =>
spark.sql("""
with
t1 as (select 1),
t2 as (select 1),
t3 as (select 1),
t4 as (select 1),
t5 as (select 1),
t6 as (select 1),
t7 as (select 1),
t8 as (select 1),
t9 as (select 1)
select * from t1, t2, t3, t4, t5, t6, t7, t8, t9""").queryExecution.assertAnalyzed()
}
// Exiting paste mode, now interpreting.
test: (i: Int)Unit
scala> test(10000)
scala> println(org.apache.spark.sql.catalyst.rules.RuleExecutor.dumpTimeSpent)
```
The result before this patch
```
Rule Effective Time / Total Time Effective Runs / Total Runs
CTESubstitution 3328796344 / 3924576425 10000 / 20000
```
The result after this patch
```
Rule Effective Time / Total Time Effective Runs / Total Runs
CTESubstitution 1503085936 / 2091992092 10000 / 20000
```
About 2 times faster.
Closes#28407 from cloud-fan/cte.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Because of ebc8fa50d0 and beec8d535f, the SQL output strings for date/timestamp - interval operation will have a malformed format, such as `struct<dateval:date,dateval + (- INTERVAL '2 years 2 months').....`
This PR restore this behavior by adding one `RuntimeReplaceable `implementation for both of the operations to have their pretty SQL strings back.
### Why are the changes needed?
restore the SQL string for datetime operations
### Does this PR introduce any user-facing change?
NO, we are restoring here
### How was this patch tested?
added unit tests
Closes#28402 from yaooqinn/SPARK-31586-F.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
- Check all available legacy formats in the tests added by https://github.com/apache/spark/pull/28345
- Check dates rebasing in legacy parsers for only one direction either days -> string or string -> days.
### Why are the changes needed?
Round trip tests can hide issues in dates rebasing. For example, if we remove rebasing from legacy parsers (from `parse()` and `format()`) the tests will pass.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By running `DateFormatterSuite`.
Closes#28398 from MaxGekk/test-rebasing-in-legacy-date-formatter.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR intends to update `sql` in `Rand`/`Randn` with no argument to make a column name deterministic.
Before this PR (a column name changes run-by-run):
```
scala> sql("select rand()").show()
+-------------------------+
|rand(7986133828002692830)|
+-------------------------+
| 0.9524061403696937|
+-------------------------+
```
After this PR (a column name fixed):
```
scala> sql("select rand()").show()
+------------------+
| rand()|
+------------------+
|0.7137935639522275|
+------------------+
// If a seed given, it is still shown in a column name
// (the same with the current behaviour)
scala> sql("select rand(1)").show()
+------------------+
| rand(1)|
+------------------+
|0.6363787615254752|
+------------------+
// We can still check a seed in explain output:
scala> sql("select rand()").explain()
== Physical Plan ==
*(1) Project [rand(-2282124938778456838) AS rand()#0]
+- *(1) Scan OneRowRelation[]
```
Note: This fix comes from #28194; the ongoing PR tests the output schema of expressions, so their schemas must be deterministic for the tests.
### Why are the changes needed?
To make output schema deterministic.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Added unit tests.
Closes#28392 from maropu/SPARK-31594.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR addresses two things:
- `SHOW TBLPROPERTIES` should supports view (a regression introduced by #26921)
- `SHOW TBLPROPERTIES` on a temporary view should return empty result (2.4 behavior instead of throwing `AnalysisException`.
### Why are the changes needed?
It's a bug.
### Does this PR introduce any user-facing change?
Yes, now `SHOW TBLPROPERTIES` works on views:
```
scala> sql("CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1")
scala> sql("SHOW TBLPROPERTIES view").show(truncate=false)
+---------------------------------+-------------+
|key |value |
+---------------------------------+-------------+
|view.catalogAndNamespace.numParts|2 |
|view.query.out.col.0 |c1 |
|view.query.out.numCols |1 |
|p2 |v2 |
|view.catalogAndNamespace.part.0 |spark_catalog|
|p1 |v1 |
|view.catalogAndNamespace.part.1 |default |
+---------------------------------+-------------+
```
And for a temporary view:
```
scala> sql("CREATE TEMPORARY VIEW tview TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1")
scala> sql("SHOW TBLPROPERTIES tview").show(truncate=false)
+---+-----+
|key|value|
+---+-----+
+---+-----+
```
### How was this patch tested?
Added tests.
Closes#28375 from imback82/show_tblproperties_followup.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
With suggestion from cloud-fan https://github.com/apache/spark/pull/28222#issuecomment-620586933
I Checked with both Presto and PostgresSQL, one is implemented intervals with ANSI style year-month/day-time, and the other is mixed and Non-ANSI. They both add the exceeded days in interval time part to the total days of the operation which extracts day from interval values.
```sql
presto> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:00' as timestamp)));
_col0
-------
14
(1 row)
Query 20200428_135239_00000_ahn7x, FINISHED, 1 node
Splits: 17 total, 17 done (100.00%)
0:01 [0 rows, 0B] [0 rows/s, 0B/s]
presto> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:01' as timestamp)));
_col0
-------
13
(1 row)
Query 20200428_135246_00001_ahn7x, FINISHED, 1 node
Splits: 17 total, 17 done (100.00%)
0:00 [0 rows, 0B] [0 rows/s, 0B/s]
presto>
```
```sql
postgres=# SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:00' as timestamp)));
date_part
-----------
14
(1 row)
postgres=# SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:01' as timestamp)));
date_part
-----------
13
```
```
spark-sql> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:01' as timestamp)));
0
spark-sql> SELECT EXTRACT(DAY FROM (cast('2020-01-15 00:00:00' as timestamp) - cast('2020-01-01 00:00:00' as timestamp)));
0
```
In ANSI standard, the day is exact 24 hours, so we don't need to worry about the conceptual day for interval extraction. The meaning of the conceptual day only takes effect when we add it to a zoned timestamp value.
### Why are the changes needed?
Both satisfy the ANSI standard and common use cases in modern SQL platforms
### Does this PR introduce any user-facing change?
No, it new in 3.0
### How was this patch tested?
add more uts
Closes#28396 from yaooqinn/SPARK-31597.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to simplify the code of `InSet.sql` and create `Literal` instances directly from Catalyst's internal values by using the default `Literal` constructor.
### Why are the changes needed?
This simplifies code and avoids unnecessary conversions to external types.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By existing test `SPARK-31563: sql of InSet for UTF8String collection` in `ColumnExpressionSuite`.
Closes#28399 from MaxGekk/fix-InSet-sql-followup.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
The implementation of TimeSub for the operation of timestamp subtracting interval is almost repetitive with TimeAdd. We can replace it with TimeAdd(l, -r) since there are equivalent.
Suggestion from https://github.com/apache/spark/pull/28310#discussion_r414259239
Besides, the Coercion rules for TimeAdd/TimeSub(date, interval) are useless anymore, so remove them in this PR since they are touched in this PR.
### Why are the changes needed?
remove redundant and useless code for easy maintenance
### Does this PR introduce any user-facing change?
Yes, the SQL string of `datetime - interval` become `datetime + (- interval)`
### How was this patch tested?
modified existing unit tests.
Closes#28381 from yaooqinn/SPARK-31586.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Add a new logical node AggregateWithHaving, and the parser should create this plan for HAVING. The analyzer resolves it to Filter(..., Aggregate(...)).
### Why are the changes needed?
The SQL parser in Spark creates Filter(..., Aggregate(...)) for the HAVING query, and Spark has a special analyzer rule ResolveAggregateFunctions to resolve the aggregate functions and grouping columns in the Filter operator.
It works for simple cases in a very tricky way as it relies on rule execution order:
1. Rule ResolveReferences hits the Aggregate operator and resolves attributes inside aggregate functions, but the function itself is still unresolved as it's an UnresolvedFunction. This stops resolving the Filter operator as the child Aggrege operator is still unresolved.
2. Rule ResolveFunctions resolves UnresolvedFunction. This makes the Aggrege operator resolved.
3. Rule ResolveAggregateFunctions resolves the Filter operator if its child is a resolved Aggregate. This rule can correctly resolve the grouping columns.
In the example query, I put a CAST, which needs to be resolved by rule ResolveTimeZone, which runs after ResolveAggregateFunctions. This breaks step 3 as the Aggregate operator is unresolved at that time. Then the analyzer starts next round and the Filter operator is resolved by ResolveReferences, which wrongly resolves the grouping columns.
See the demo below:
```
SELECT SUM(a) AS b, '2020-01-01' AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10
```
The query's result is
```
+---+----------+
| b| fake|
+---+----------+
| 2|2020-01-01|
+---+----------+
```
But if we add CAST, it will return an empty result.
```
SELECT SUM(a) AS b, CAST('2020-01-01' AS DATE) AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10
```
### Does this PR introduce any user-facing change?
Yes, bug fix for cast in having aggregate expressions.
### How was this patch tested?
New UT added.
Closes#28294 from xuanyuanking/SPARK-31519.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.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/28318, to make the code more readable, by adding some comments to explain the trick and simplify the code to use a boolean flag instead of 2 string sets.
This PR also fixes various problems:
1. the name check should consider case sensitivity
2. forward name conflicts like `with t as (with t2 as ...), t2 as ...` is not a real conflict and we shouldn't fail.
### Why are the changes needed?
correct the behavior
### Does this PR introduce any user-facing change?
yes, fix the fore-mentioned behaviors.
### How was this patch tested?
new tests
Closes#28371 from cloud-fan/followup.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
Remove all the extra whitespaces in the formatted explain.
### Why are the changes needed?
The number of extra whitespaces of the formatted explain becomes different between master and branch-3.0. This causes a problem that whenever we backport formatted explain related tests from master to branch-3.0, it will fail branch-3.0. Besides, extra whitespaces are always disallowed in Spark. Thus, we should remove them as possible as we can.
### Does this PR introduce any user-facing change?
No, formatted explain is newly added in Spark 3.0.
### How was this patch tested?
Updated sql query tests.
Closes#28315 from Ngone51/fix_extra_spaces.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
To follow ANSI,the expressions - `date + interval`, `interval + date` and `date - interval` should only accept intervals which the `microseconds` part is 0.
### Why are the changes needed?
Better ANSI compliance
### Does this PR introduce any user-facing change?
No, this PR should target 3.0.0 in which this feature is newly added.
### How was this patch tested?
add more unit tests
Closes#28310 from yaooqinn/SPARK-31527.
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 modifies LegacyDateFormatter#parse to return proleptic Gregorian days rather than hybrid Julian days.
### Why are the changes needed?
The legacy time parser currently returns epoch days in the hybrid Julian calendar. However, the callers to the legacy parser (e.g., UnivocityParser, JacksonParser) expect epoch days in the proleptic Gregorian calendar. As a result, pre-Gregorian dates like '1000-01-01' get interpreted as '1000-01-06'.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Manual testing and modified existing unit tests.
Closes#28345 from bersprockets/SPARK-31557.
Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR fixes a CTE substitution issue so as to the following SQL return the correct empty result:
```
WITH t(c) AS (SELECT 1)
SELECT * FROM t
WHERE c IN (
WITH t(c) AS (SELECT 2)
SELECT * FROM t
)
```
Before this PR the result was `1`.
### Why are the changes needed?
To fix a correctness issue.
### Does this PR introduce any user-facing change?
Yes, fixes a correctness issue.
### How was this patch tested?
Added new test case.
Closes#28318 from peter-toth/SPARK-31535-fix-nested-cte-substitution.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
In the PR, I propose to fix the `InSet.sql` method for the cases when input collection contains values of internal Catalyst's types, for instance `UTF8String`. Elements of the input set `hset` are converted to Scala types, and wrapped by `Literal` to properly form SQL view of the input collection.
### Why are the changes needed?
The changes fixed the bug in `InSet.sql` that makes wrong assumption about types of collection elements. See more details in SPARK-31563.
### Does this PR introduce any user-facing change?
Highly likely, not.
### How was this patch tested?
Added a test to `ColumnExpressionSuite`
Closes#28343 from MaxGekk/fix-InSet-sql.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
the 2 method `arrayClassFor` and `dataTypeFor` in `ScalaReflection` call each other circularly, the cases in `dataTypeFor` are not fully handled in `arrayClassFor`
For example:
```scala
scala> implicit def newArrayEncoder[T <: Array[_] : TypeTag]: Encoder[T] = ExpressionEncoder()
newArrayEncoder: [T <: Array[_]](implicit evidence$1: reflect.runtime.universe.TypeTag[T])org.apache.spark.sql.Encoder[T]
scala> val decOne = Decimal(1, 38, 18)
decOne: org.apache.spark.sql.types.Decimal = 1E-18
scala> val decTwo = Decimal(2, 38, 18)
decTwo: org.apache.spark.sql.types.Decimal = 2E-18
scala> val decSpark = Array(decOne, decTwo)
decSpark: Array[org.apache.spark.sql.types.Decimal] = Array(1E-18, 2E-18)
scala> Seq(decSpark).toDF()
java.lang.ClassCastException: org.apache.spark.sql.types.DecimalType cannot be cast to org.apache.spark.sql.types.ObjectType
at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$arrayClassFor$1(ScalaReflection.scala:131)
at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:879)
at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:878)
at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:49)
at org.apache.spark.sql.catalyst.ScalaReflection$.arrayClassFor(ScalaReflection.scala:120)
at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$dataTypeFor$1(ScalaReflection.scala:105)
at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:879)
at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:878)
at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:49)
at org.apache.spark.sql.catalyst.ScalaReflection$.dataTypeFor(ScalaReflection.scala:88)
at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$serializerForType$1(ScalaReflection.scala:399)
at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:879)
at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:878)
at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:49)
at org.apache.spark.sql.catalyst.ScalaReflection$.serializerForType(ScalaReflection.scala:393)
at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:57)
at newArrayEncoder(<console>:57)
... 53 elided
scala>
```
In this PR, we add the missing cases to `arrayClassFor`
### Why are the changes needed?
bugfix as described above
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
add a test for array encoders
Closes#28324 from yaooqinn/SPARK-31552.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
`LIKE ANY/SOME` and `LIKE ALL` operators are mostly used when we are matching a text field with numbers of patterns. For example:
Teradata / Hive 3.0 / Snowflake:
```sql
--like any
select 'foo' LIKE ANY ('%foo%','%bar%');
--like all
select 'foo' LIKE ALL ('%foo%','%bar%');
```
PostgreSQL:
```sql
-- like any
select 'foo' LIKE ANY (array['%foo%','%bar%']);
-- like all
select 'foo' LIKE ALL (array['%foo%','%bar%']);
```
This PR add support these two operators.
More details:
https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/4~AyrPNmDN0Xk4SALLo6aQhttps://issues.apache.org/jira/browse/HIVE-15229https://docs.snowflake.net/manuals/sql-reference/functions/like_any.html
### Why are the changes needed?
To smoothly migrate SQLs to Spark SQL.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Unit test.
Closes#27477 from wangyum/SPARK-30724.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
1. Modified `ParquetFilters.valueCanMakeFilterOn()` to accept filters with `java.time.LocalDate` attributes.
2. Modified `ParquetFilters.dateToDays()` to support both types `java.sql.Date` and `java.time.LocalDate` in conversions to days.
3. Add implicit conversion from `LocalDate` to `Expression` (`Literal`).
### Why are the changes needed?
To support pushed down filters with `java.time.LocalDate` attributes. Before the changes, date filters are not pushed down to Parquet datasource when `spark.sql.datetime.java8API.enabled` is `true`.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Added a test to `ParquetFilterSuite`
Closes#28259 from MaxGekk/parquet-filter-java8-date-time.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR intends to add a new test suite for `ExpressionInfo`. Major changes are as follows;
- Added a new test suite named `ExpressionInfoSuite`
- To improve test coverage, added a test for error handling in `ExpressionInfoSuite`
- Moved the `ExpressionInfo`-related tests from `UDFSuite` to `ExpressionInfoSuite`
- Moved the related tests from `SQLQuerySuite` to `ExpressionInfoSuite`
- Added a comment in `ExpressionInfoSuite` (followup of https://github.com/apache/spark/pull/28224)
### Why are the changes needed?
To improve test suites/coverage.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Added tests.
Closes#28308 from maropu/SPARK-31526.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Override the canonicalized fields with respect to the result of `needsTimeZone`.
### Why are the changes needed?
The current approach breaks sematic equal of two cast expressions that don't relate with datetime type. If we don't need to use `timeZone` information casting `from` type to `to` type, then the timeZoneId should not influence the canonicalize result.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
New UT added.
Closes#28288 from xuanyuanking/SPARK-31515.
Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
\_FUNC\_ is used in note() of `ExpressionDescription` since https://github.com/apache/spark/pull/28248, it can be more cases later, we should replace it with function name for documentation
### Why are the changes needed?
doc fix
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
pass Jenkins, and verify locally with Jekyll serve
Closes#28305 from yaooqinn/SPARK-31474-F.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
- Document row field values of `DATE` and `TIMESTAMP` type returned by `Row.get()` and `Row.apply`.
- Refer to `Row.get()` from the description of filter values
### Why are the changes needed?
Reflect current behaviour of Row's method `apply()` and `get()` in comments to inform users about different return types that are depended on the SQL config settings `spark.sql.datetime.java8API.enabled`.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Run `$ ./dev/scalastyle`
Closes#28300 from MaxGekk/doc-filter-date-time.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Extracting millennium, century, decade, millisecond, microsecond and epoch from datetime is neither ANSI standard nor quite common in modern SQL platforms. Most of the systems listing below does not support these except PostgreSQL and redshift.
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDFhttps://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htmhttps://prestodb.io/docs/current/functions/datetime.htmlhttps://docs.cloudera.com/documentation/enterprise/5-8-x/topics/impala_datetime_functions.htmlhttps://docs.snowflake.com/en/sql-reference/functions-date-time.html#label-supported-date-time-partshttps://www.postgresql.org/docs/9.1/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT
This PR removes these extract fields support from extract function for date and timestamp values
`isoyear` is PostgreSQL specific but `yearofweek` is more commonly used across platforms
`isodow` is PostgreSQL specific but `iso` as a suffix is more commonly used across platforms so, `dow_iso` and `dayofweek_iso` is used to replace it.
For historical reasons, we have [`dayofweek`, `dow`] implemented for representing a non-ISO day-of-week and a newly added `isodow` from PostgreSQL for ISO day-of-week. Many other systems only have one week-numbering system support and use either full names or abbreviations. Things in spark become a little bit complicated.
1. because of the existence of `isodow`, so we need to add iso-prefix to `dayofweek` to make a pair for it too. [`dayofweek`, `isodayofweek`, `dow` and `isodow`]
2. because there are rare `iso`-prefixed systems and more systems choose `iso`-suffixed way, so we may result in [`dayofweek`, `dayofweekiso`, `dow`, `dowiso`]
3. `dayofweekiso` looks nice and has use cases in the platforms listed above, e.g. snowflake, but `dowiso` looks weird and no use cases found.
4. with a discussion the community,we have agreed with an underscore before `iso` may look much better because `isodow` is new and there is no standard for `iso` kind of things, so this may be good for us to make it simple and clear for end-users if they are well documented too.
Thus, we finally result in [`dayofweek`, `dow`] for Non-ISO day-of-week system and [`dayofweek_iso`, `dow_iso`] for ISO system
### Why are the changes needed?
Remove some nonstandard and uncommon features as we can add them back if necessary
### Does this PR introduce any user-facing change?
NO, we should target this to 3.0.0 and these are added during 3.0.0
### How was this patch tested?
Remove unused tests
Closes#28284 from yaooqinn/SPARK-31507.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
override the `sql` method of `StringTrim`, `StringTrimLeft` and `StringTrimRight`, to use the standard SQL syntax.
### Why are the changes needed?
The current implementation is wrong. It gives you a SQL string that returns different result.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
new tests
Closes#28281 from cloud-fan/sql.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
```sql
spark-sql> SELECT extract(dayofweek from '2009-07-26');
1
spark-sql> SELECT extract(dow from '2009-07-26');
0
spark-sql> SELECT extract(isodow from '2009-07-26');
7
spark-sql> SELECT dayofweek('2009-07-26');
1
spark-sql> SELECT weekday('2009-07-26');
6
```
Currently, there are 4 types of day-of-week range:
1. the function `dayofweek`(2.3.0) and extracting `dayofweek`(2.4.0) result as of Sunday(1) to Saturday(7)
2. extracting `dow`(3.0.0) results as of Sunday(0) to Saturday(6)
3. extracting` isodow` (3.0.0) results as of Monday(1) to Sunday(7)
4. the function `weekday`(2.4.0) results as of Monday(0) to Sunday(6)
Actually, extracting `dayofweek` and `dow` are both derived from PostgreSQL but have different meanings.
https://issues.apache.org/jira/browse/SPARK-23903https://issues.apache.org/jira/browse/SPARK-28623
In this PR, we make extracting `dow` as same as extracting `dayofweek` and the `dayofweek` function for historical reason and not breaking anything.
Also, add more documentation to the extracting function to make extract field more clear to understand.
### Why are the changes needed?
Consistency insurance
### Does this PR introduce any user-facing change?
yes, doc updated and extract `dow` is as same as `dayofweek`
### How was this patch tested?
1. modified ut
2. local SQL doc verification
#### before
![image](https://user-images.githubusercontent.com/8326978/79601949-3535b100-811c-11ea-957b-a33d68641181.png)
#### after
![image](https://user-images.githubusercontent.com/8326978/79601847-12a39800-811c-11ea-8ff6-aa329255d099.png)
Closes#28248 from yaooqinn/SPARK-31474.
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 skips creating the partition specs in `ShufflePartitionsUtil` for 0-size partitions, which avoids launching unnecessary tasks that do nothing.
### Why are the changes needed?
launching tasks that do nothing is a waste.
### Does this PR introduce any user-facing change?
no
### How was this patch tested?
updated tests
Closes#28226 from cloud-fan/aqe.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR is the follow-up PR of https://github.com/apache/spark/pull/28003
- add a migration guide
- add an end-to-end test case.
### Why are the changes needed?
The original PR made the major behavior change in the user-facing RESET command.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Added a new end-to-end test
Closes#28265 from gatorsmile/spark-31234followup.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
### What changes were proposed in this pull request?
1. Generate rebasing arrays for micros up to 2037 in `RebaseDateTimeSuite.generateRebaseJson()`.
2. Exclude 4 time zones from the black list in `generateRebaseJson()`.
3. Re-generate JSON files with rebasing info - `gregorian-julian-rebase-micros.json` and `julian-gregorian-rebase-micros.json`.
### Why are the changes needed?
1. `sun.util.calendar.ZoneInfo` resolves DST after 2037 year incorrectly. See aa318070b2/jdk/src/share/classes/sun/util/calendar/ZoneInfo.java (L55-L62) . By restricting the rebase arrays to 2037 year, we follow the behaviour of `ZoneInfo` which uses DST of 2037 for all years beyond 2037.
2. To enable optimization of micros rebasing via switch arrays for the time zones:
- Asia/Tehran
- Iran
- Africa/Casablanca
- Africa/El_Aaiun
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
By existing test suites `RebaseDateTimeUtils`, `DateTimeUtilsSuite` and `DateFunctionsSuite`.
Closes#28253 from MaxGekk/fix-4-time-zones-rebasing.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR intends to add an ExpressionInfo entry for EXTRACT for better documentations.
This PR comes from the comment in https://github.com/apache/spark/pull/21479#discussion_r409900080
### Why are the changes needed?
To make SQL documentations complete.
### Does this PR introduce any user-facing change?
Yes, this PR updates the `Spark SQL, Built-in Functions` page.
### How was this patch tested?
Run the example tests.
Closes#28251 from maropu/AddExtractExpr.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This PR is to dump the codegen and compilation time for benchmark query tests.
### Why are the changes needed?
Measure the codegen and compilation time costs in TPC-DS queries
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Manual test in my local laptop:
```
23:13:12.845 WARN org.apache.spark.sql.TPCDSQuerySuite:
=== Metrics of Whole-stage Codegen ===
Total code generation time: 21.275102261 seconds
Total compilation time: 12.223771828 seconds
```
Closes#28252 from gatorsmile/testMastercode.
Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This PR intends to fix a bug that occurs when comparing null types to decimal types in master/branch-3.0;
```
scala> Seq(BigDecimal(10)).toDF("v1").selectExpr("v1 = NULL").explain(true)
org.apache.spark.sql.AnalysisException: cannot resolve '(`v1` = NULL)' due to data type mismatch: differing types in '(`v1` = NULL)' (decimal(38,18) and null).; line 1 pos 0;
'Project [(v1#5 = null) AS (v1 = NULL)#7]
+- Project [value#2 AS v1#5]
+- LocalRelation [value#2]
...
```
The query above passed in v2.4.5.
### Why are the changes needed?
bugfix
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Added tests.
Closes#28241 from maropu/SPARK-31468.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Currently, we can extract `millennium/century/decade/year/quarter/month/week/day/hour/minute/second(with fractions)//millisecond/microseconds` and `epoch` from interval values
While getting the `millennium/century/decade/year`, it means how many the interval `months` part can be converted to that unit-value. The content of `millennium/century/decade` will overlap `year` and each other.
While getting `month/day` and so on, it means the integral remainder of the previous unit. Here all the units including `year` are individual.
So while extracting `year`, `month`, `day`, `hour`, `minute`, `second`, which are ANSI primary datetime units, the semantic is `extracting`, but others might refer to `transforming`.
While getting epoch we have treat month as 30 days which varies the natural Calendar rules we use.
To avoid ambiguity, I suggest we should only support those extract field defined ANSI with their abbreviations.
### Why are the changes needed?
Extracting `millennium`, `century` etc does not obey the meaning of extracting, and they are not so useful and worth maintaining.
The `extract` is ANSI standard expression and `date_part` is its pg-specific alias function. The current support extract-fields are fully bought from PostgreSQL.
With a look at other systems like Presto/Hive, they don't support those ambiguous fields too.
e.g. Hive 2.2.x also take it from PostgreSQL but without introducing those ambiguous fields https://issues.apache.org/jira/secure/attachment/12828349/HIVE-14579
e.g. presto
```sql
presto> select extract(quater from interval '10-0' year to month);
Query 20200417_094723_00020_m8xq4 failed: line 1:8: Invalid EXTRACT field: quater
select extract(quater from interval '10-0' year to month)
presto> select extract(decade from interval '10-0' year to month);
Query 20200417_094737_00021_m8xq4 failed: line 1:8: Invalid EXTRACT field: decade
select extract(decade from interval '10-0' year to month)
```
### Does this PR introduce any user-facing change?
Yes, as we already have previews versions, this PR will remove support for extracting `millennium/century/decade/quarter/week/millisecond/microseconds` and `epoch` from intervals with `date_part` function
### How was this patch tested?
rm some used tests
Closes#28242 from yaooqinn/SPARK-31469.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
I checked all the config of Spark again. find some new commit not add version information.
**Test.scala**
Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.testing.skipValidateCores | 3.1.0 | SPARK-29154 | 474b1bb5c2bce2f83c4dd8e19b9b7c5b3aebd6c4#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |
**SQL**
Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.legacy.integerGroupingId | 3.1.0 | SPARK-30279 | 71c73d58f6e88d2558ed2e696897767d93bac60f#diff-9a6b543db706f1a90f790783d6930a13 |
The two config only exists in branch master.
### Why are the changes needed?
Supplement version information.
### Does this PR introduce any user-facing change?
'No'.
### How was this patch tested?
Jenkins test.
Closes#28233 from beliefer/sql-conf-version-legacy-integerGroupingId.
Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Make `UnsafeKVExternalSorter` / `VariableLengthRowBasedKeyValueBatch ` also respect `UnsafeAlignedOffset` when reading the record and update some out of date comemnts.
### Why are the changes needed?
Since `BytesToBytesMap` respects `UnsafeAlignedOffset` when writing the record, `UnsafeKVExternalSorter` should also respect `UnsafeAlignedOffset` when reading the record from `BytesToBytesMap` otherwise it will causes data correctness issue.
Unlike `UnsafeKVExternalSorter` may reading records from `BytesToBytesMap`, `VariableLengthRowBasedKeyValueBatch` writes and reads records by itself. Thus, similar to #22053 and [comment](https://github.com/apache/spark/pull/22053#issuecomment-411975239) there, fix for `VariableLengthRowBasedKeyValueBatch` more likely an improvement for the support of SPARC platform.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Manually tested `HashAggregationQueryWithControlledFallbackSuite` with `UAO_SIZE=8` to simulate SPARC platform. And tests only pass with this fix.
Closes#28195 from Ngone51/fix_uao.
Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR moves the `ExpressionEncoder.toRow` and `ExpressionEncoder.fromRow` functions into their own function objects(`ExpressionEncoder.Serializer` & `ExpressionEncoder.Deserializer`). This effectively makes the `ExpressionEncoder` stateless, thread-safe and (more) reusable. The function objects are not thread safe, however they are documented as such and should be used in a more limited scope (making it easier to reason about thread safety).
### Why are the changes needed?
ExpressionEncoders are not thread-safe. We had various (nasty) bugs because of this.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests.
Closes#28223 from hvanhovell/SPARK-31450.
Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
SPARK-21870 (cb0cddf#diff-06dc5de6163687b7810aa76e7e152a76R146-R149) caused significant performance regression in cases where the source code size is fairly large as `HashAggregateExec` uses `Block.length` to decide on splitting the code. The change in `length` makes sense as the comment and extra new lines shouldn't be taken into account when deciding on splitting, but the regular expression based approach is very slow and adds a big relative overhead to cases where the execution is quick (small number of rows).
This PR:
- restores `Block.length` to its original form
- places comments in `HashAggragateExec` with `CodegenContext.registerComment` so as to appear only when comments are enabled (`spark.sql.codegen.comments=true`)
Before this PR:
```
deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
250 deep x 400 rows (read in-mem) 1137 1143 8 0.1 11368.3 0.0X
```
After this PR:
```
deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
250 deep x 400 rows (read in-mem) 167 180 7 0.6 1674.3 0.1X
```
### Why are the changes needed?
To fix performance regression.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#28083 from peter-toth/SPARK-30564-use-comment-placeholders.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
In the PR, I propose to change rebasing of not-existed timestamps in the hybrid calendar (Julian + Gregorian since 1582-10-15) in the range [1582-10-05, 1582-10-15). Not existed timestamps from the range are shifted to the first valid date in the hybrid calendar - 1582-10-15. The changes affect only `rebaseGregorianToJulianMicros()` because reverse rebasing from the hybrid timestamps to Proleptic Gregorian timestamps does not have such problem.
The shifting affects only the date part of timestamps while keeping the time part as is. For example:
```
1582-10-10 00:11:22.334455 -> 1582-10-15 00:11:22.334455
```
### Why are the changes needed?
Currently, not-existed timestamps are shifted by standard difference between Julian and Gregorian calendar on 1582-10-04, for example 1582-10-14 00:00:00 -> 1582-10-24 00:00:00. That contradicts to shifting of not existed dates in other cases, for example:
```
scala> sql("select timestamp'1990-9-31 12:12:12'").show
+----------------------------------+
|TIMESTAMP('1990-10-01 12:12:12.0')|
+----------------------------------+
| 1990-10-01 12:12:12|
+----------------------------------+
```
### Does this PR introduce any user-facing change?
Yes, this impacts on conversion of Spark SQL `TIMESTAMP` values to external timestamps based on non-Proleptic Gregorian calendar. For example, while saving the 1582-10-14 12:13:14 date to ORC files, it will be shifted to the next valid date 1582-10-15 12:13:14.
### How was this patch tested?
- Added tests to `RebaseDateTimeSuite` and to `OrcSourceSuite`
- By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`, `CollectionExpressionsSuite`, `ParquetIOSuite`.
Closes#28227 from MaxGekk/fix-not-exist-timestamps.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to change rebasing of not-existed dates in the hybrid calendar (Julian + Gregorian since 1582-10-15) in the range (1582-10-04, 1582-10-15). Not existed dates from the range are shifted to the first valid date in the hybrid calendar - 1582-10-15. The changes affect only `rebaseGregorianToJulianDays()` because reverse rebasing from the hybrid dates to Proleptic Gregorian dates does not have such problem.
### Why are the changes needed?
Currently, not-existed dates are shifted by standard difference between Julian and Gregorian calendar on 1582-10-04, for example 1582-10-14 -> 1582-10-24. That's contradict to shifting not existed dates in other cases, for example:
```
scala> sql("select date'1990-9-31'").show
+-----------------+
|DATE '1990-10-01'|
+-----------------+
| 1990-10-01|
+-----------------+
```
### Does this PR introduce any user-facing change?
Yes, this impacts on conversion of Spark SQL `DATE` values to external dates based on non-Proleptic Gregorian calendar. For example, while saving the 1582-10-14 date to ORC files, it will be shifted to the next valid date 1582-10-15.
### How was this patch tested?
- Added tests to `RebaseDateTimeSuite` and to `OrcSourceSuite`
- By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`, `CollectionExpressionsSuite`, `ParquetIOSuite`.
Closes#28225 from MaxGekk/fix-not-exist-dates.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Optimise the `toJavaDate()` method of `DateTimeUtils` by:
1. Re-using `rebaseGregorianToJulianDays` optimised by #28067
2. Creating `java.sql.Date` instances from milliseconds in UTC since the epoch instead of date-time fields. This allows to avoid "normalization" inside of `java.sql.Date`.
Also new benchmark for collecting dates is added to `DateTimeBenchmark`.
### Why are the changes needed?
The changes fix the performance regression of collecting `DATE` values comparing to Spark 2.4 (see `DateTimeBenchmark` in https://github.com/MaxGekk/spark/pull/27):
Spark 2.4.6-SNAPSHOT:
```
To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date 559 603 38 8.9 111.8 1.0X
Collect dates 2306 3221 1558 2.2 461.1 0.2X
```
Before the changes:
```
To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date 1052 1130 73 4.8 210.3 1.0X
Collect dates 3251 4943 1624 1.5 650.2 0.3X
```
After:
```
To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date 416 419 3 12.0 83.2 1.0X
Collect dates 1928 2759 1180 2.6 385.6 0.2X
```
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
- By existing tests suites, in particular, `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`.
- Re-run `DateTimeBenchmark` in the environment:
| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |
Closes#28212 from MaxGekk/optimize-toJavaDate.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to re-use optimized implementation of days rebase function `rebaseJulianToGregorianDays()` introduced by the PR #28067 in conversion of `java.sql.Date` values to Catalyst's `DATE` values. The function `fromJavaDate` in `DateTimeUtils` was re-written by taking the implementation from Spark 2.4, and by rebasing the final results via `rebaseJulianToGregorianDays()`.
Also I updated `DateTimeBenchmark`, and added a benchmark for conversion from `java.sql.Date`.
### Why are the changes needed?
The PR fixes the regression of parallelizing a collection of `java.sql.Date` values, and improves performance of converting external values to Catalyst's `DATE` values:
- x4 on the master branch
- 30% against Spark 2.4.6-SNAPSHOT
Spark 2.4.6-SNAPSHOT:
```
To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date 614 655 43 8.1 122.8 1.0X
```
Before the changes:
```
To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date 1154 1206 46 4.3 230.9 1.0X
```
After:
```
To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Date 427 434 7 11.7 85.3 1.0X
```
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
- By existing tests suites, in particular, `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite`.
- Re-run `DateTimeBenchmark` in the environment:
| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |
Closes#28205 from MaxGekk/optimize-fromJavaDate.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Refine the code comments of days rebasing, to be consistent with the micros rebasing. i.e. one method is the actual implementation and the other variant is the optimized version.
### Why are the changes needed?
improve code comments
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
N/A
Closes#28199 from cloud-fan/comment.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Reuse the `rebaseGregorianToJulianMicros()` and `rebaseJulianToGregorianMicros()` functions introduced by the PR #28119 in `DateTimeUtils`.`toJavaTimestamp()` and `fromJavaTimestamp()`. Actually, new implementation is derived from Spark 2.4 + rebasing via pre-calculated rebasing maps.
### Why are the changes needed?
The changes speed up conversions to/from java.sql.Timestamp, and as a consequence the PR improve performance of ORC datasource in loading/saving timestamps:
- Saving ~ **x2.8 faster** in master, and -11% against Spark 2.4.6
- Loading - **x3.2-4.5 faster** in master, -5% against Spark 2.4.6
Before:
```
Save timestamps to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
after 1582 59877 59877 0 1.7 598.8 0.0X
before 1582 61361 61361 0 1.6 613.6 0.0X
Load timestamps from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off 48197 48288 118 2.1 482.0 1.0X
after 1582, vec on 38247 38351 128 2.6 382.5 1.3X
before 1582, vec off 53179 53359 249 1.9 531.8 0.9X
before 1582, vec on 44076 44268 269 2.3 440.8 1.1X
```
After:
```
Save timestamps to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
after 1582 21250 21250 0 4.7 212.5 0.1X
before 1582 22105 22105 0 4.5 221.0 0.1X
Load timestamps from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off 14903 14933 40 6.7 149.0 1.0X
after 1582, vec on 8342 8426 73 12.0 83.4 1.8X
before 1582, vec off 15528 15575 76 6.4 155.3 1.0X
before 1582, vec on 9025 9075 61 11.1 90.2 1.7X
```
Spark 2.4.6-SNAPSHOT:
```
Save timestamps to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
after 1582 18858 18858 0 5.3 188.6 1.0X
before 1582 18508 18508 0 5.4 185.1 1.0X
Load timestamps from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
after 1582, vec off 14063 14177 143 7.1 140.6 1.0X
after 1582, vec on 5955 6029 100 16.8 59.5 2.4X
before 1582, vec off 14119 14126 7 7.1 141.2 1.0X
before 1582, vec on 5991 6007 25 16.7 59.9 2.3X
```
### Does this PR introduce any user-facing change?
Yes, the `to_utc_timestamp` function returns the later local timestamp in the case of overlapping local timestamps at daylight saving time. it's changed back to the 2.4 behavior.
### How was this patch tested?
- By existing test suite `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuites`, `ParquetIOSuite`, `OrcHadoopFsRelationSuite`.
- Re-generating results of the benchmarks `DateTimeBenchmark` and `DateTimeRebaseBenchmark` in the environment:
| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_242 and OpenJDK 64-Bit Server VM 11.0.6+10 |
Closes#28189 from MaxGekk/optimize-to-from-java-timestamp.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>