Commit graph

9651 commits

Author SHA1 Message Date
Jungtaek Lim (HeartSaVioR) ac6406e757 [SPARK-31831][SQL] HiveSessionImplSuite flakiness fix via mocking instances earlier than initializing HiveSessionImpl
### What changes were proposed in this pull request?

This patch changes the HiveSessionImplSuite to mock instances "before" initializing HiveSessionImpl, to avoid possible classloader issue.

### Why are the changes needed?

The failures of HiveSessionImplSuite always come from classloader issue. While I don't have clear idea what is happening, there's no part possibly dealing with classloader, except initializing HiveSessionImpl. We can move the mock initializations earlier than initialing HiveSessionImpl so that it can avoid possible classloader issue.

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

No.

### How was this patch tested?

Verified with multiple triggers of Jenkins builds

Closes #29039 from HeartSaVioR/hive-session-impl-suite-flakiness-fix.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-09 14:32:20 -07:00
xiepengjie 523e238d2a [SPARK-32192][SQL] Print column name when throws ClassCastException
### What changes were proposed in this pull request?

When somebody changed the type of partition's field, spark will throw ClassCastException. For example, we have a table like this:
```
drop table if exists cast_exception_test;

create table cast_exception_test(c1 int, c2 string) partitioned by (dt string) stored as orc;

insert into table cast_exception_test partition(dt='2020-04-08') values('1', 'jeff_1');
```

If you change the field's type in hive, query the old partition, spark will throw ClassCastException, but hive will not:
```
-- change the field's type using hive
alter table cast_exception_test change column c1 c1 string;
-- hive correct,  but spark throws ClassCastException
select * from cast_exception_test where dt='2020-04-08';
```

### Why are the changes needed?

When the table has many fields, we don's known which field has been changed. If we print out log about this exception, it will very helpful for us to troubleshoot.

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

When the ClassCastException is caused by changed field's type, you can search which field has problem in exexutor logs:
```
20/04/09 17:22:05 ERROR hive.HadoopTableReader: Exception thrown in field <c1>
```

### How was this patch tested?

First, prepare the test data, the table is partitioned and stored as orc:
```
drop table if exists cast_exception_test;
create table cast_exception_test(c1 int, c2 string) partitioned by (dt string) stored as orc;
insert into table cast_exception_test partition(dt='2020-04-08') values('1', 'jeff_1');
```

Then, change the field's type in hive.
```
alter table cast_exception_test change column c1 c1 string;
```

Now the metadata of the table has been modified, but the partition's metadata which is stored in orc file or hive metastore's mysql is still old. So, query command throws ClassCastException in spark, because spark use table's metadata which is different from orc file's metadata. But hive use partition's metadata which is the same as orc file's metadata.

If you query the old partition, spark will thrown ClassCastException, but hive will not:
```
select * from cast_exception_test where dt='2020-04-08';
```

Closes #29010 from StefanXiepj/SPARK-32192.

Authored-by: xiepengjie <xiepengjie@didiglobal.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-07-09 09:33:54 -05:00
Erik Erlandson 1cb5bfc47a [SPARK-32159][SQL] Fix integration between Aggregator[Array[_], _, _] and UnresolvedMapObjects
Context: The fix for SPARK-27296 introduced by #25024 allows `Aggregator` objects to appear in queries. This works fine for aggregators with atomic input types, e.g. `Aggregator[Double, _, _]`.

However it can cause a null pointer exception if the input type is `Array[_]`.  This was historically considered an ignorable case for serialization of `UnresolvedMapObjects`, but the new ScalaAggregator class causes these expressions to be serialized over to executors because the resolve-and-bind is being deferred.

### What changes were proposed in this pull request?
A new rule `ResolveEncodersInScalaAgg` that performs the resolution of the expressions contained in the encoders so that properly resolved expressions are serialized over to executors.

### Why are the changes needed?
Applying an aggregator of the form `Aggregator[Array[_], _, _]` using `functions.udaf()` currently causes a null pointer error in Catalyst.

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

### How was this patch tested?
A unit test has been added that does aggregation with array types for input, buffer, and output. I have done additional testing with my own custom aggregators in the spark REPL.

Closes #28983 from erikerlandson/fix-spark-32159.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-09 08:42:20 +00:00
Jungtaek Lim (HeartSaVioR) 526cb2d1ba [SPARK-32148][SS] Fix stream-stream join issue on missing to copy reused unsafe row
### What changes were proposed in this pull request?

This patch fixes the odd join result being occurred from stream-stream join for state store format V2.

There're some spots on V2 path which leverage UnsafeProjection. As the result row is reused, the row should be copied to avoid changing value during reading (or make sure the caller doesn't affect by such behavior) but `SymmetricHashJoinStateManager.removeByValueCondition` violates the case.

This patch makes `KeyWithIndexToValueRowConverterV2.convertValue` copy the row by itself so that callers don't need to take care about it. This patch doesn't change the behavior of `KeyWithIndexToValueRowConverterV2.convertToValueRow` to avoid double-copying, as the caller is expected to store the row which the implementation of state store will call `copy()`.

This patch adds such behavior into each method doc in `KeyWithIndexToValueRowConverter`, so that further contributors can read through and make sure the change / new addition doesn't break the contract.

### Why are the changes needed?

Stream-stream join with state store format V2 (newly added in Spark 3.0.0) has a serious correctness bug which brings indeterministic result.

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

Yes, some of Spark 3.0.0 users using stream-stream join from the new checkpoint (as the bug exists to only v2 format path) may encounter wrong join result. This patch will fix it.

### How was this patch tested?

Reported case is converted to the new UT, and confirmed UT passed. All UTs in StreamingInnerJoinSuite and StreamingOuterJoinSuite passed as well

Closes #28975 from HeartSaVioR/SPARK-32148.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-09 07:37:06 +00:00
Wenchen Fan 8c5bee599d [SPARK-28067][SPARK-32018] Fix decimal overflow issues
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/27627 to fix the remaining issues. There are 2 issues fixed in this PR:
1. `UnsafeRow.setDecimal` can set an overflowed decimal and causes an error when reading it. The expected behavior is to return null.
2. The update/merge expression for decimal type in `Sum` is wrong. We shouldn't turn the `sum` value back to 0 after it becomes null due to overflow. This issue was hidden because:
2.1 for hash aggregate, the buffer is unsafe row. Due to the first bug, we fail when overflow happens, so there is no chance to mistakenly turn null back to 0.
2.2 for sort-based aggregate, the buffer is generic row. The decimal can overflow (the Decimal class has unlimited precision) and we don't have the null problem.

If we only fix the first bug, then the second bug is exposed and test fails. If we only fix the second bug, there is no way to test it. This PR fixes these 2 bugs together.

### Why are the changes needed?

Fix issues during decimal sum when overflow happens

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

Yes. Now decimal sum can return null correctly for overflow under non-ansi mode.

### How was this patch tested?

new test and updated test

Closes #29026 from cloud-fan/decimal.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:56:40 +09:00
Takuya UESHIN cfecc2030d [SPARK-32160][CORE][PYSPARK] Disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This PR proposes to disallow to create `SparkContext` in executors, e.g., in UDFs.

### Why are the changes needed?

Currently executors can create SparkContext, but shouldn't be able to create it.

```scala
sc.range(0, 1).foreach { _ =>
  new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
}
```

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

Yes, users won't be able to create `SparkContext` in executors.

### How was this patch tested?

Addes tests.

Closes #28986 from ueshin/issues/SPARK-32160/disallow_spark_context_in_executors.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:51:56 +09:00
Ryan Blue 3bb1ac597a [SPARK-32168][SQL] Fix hidden partitioning correctness bug in SQL overwrite
### What changes were proposed in this pull request?

When converting an `INSERT OVERWRITE` query to a v2 overwrite plan, Spark attempts to detect when a dynamic overwrite and a static overwrite will produce the same result so it can use the static overwrite. Spark incorrectly detects when dynamic and static overwrites are equivalent when there are hidden partitions, such as `days(ts)`.

This updates the analyzer rule `ResolveInsertInto` to always use a dynamic overwrite when the mode is dynamic, and static when the mode is static. This avoids the problem by not trying to determine whether the two plans are equivalent and always using the one that corresponds to the partition overwrite mode.

### Why are the changes needed?

This is a correctness bug. If a table has hidden partitions, all of the values for those partitions are dropped instead of dynamically overwriting changed partitions.

This only affects SQL commands (not `DataFrameWriter`) writing to tables that have hidden partitions. It is also only a problem when the partition overwrite mode is dynamic.

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

Yes, it fixes the correctness bug detailed above.

### How was this patch tested?

* This updates the in-memory table to support a hidden partition transform, `days`, and adds a test case to `DataSourceV2SQLSuite` in which the table uses this hidden partition function. This test fails without the fix to `ResolveInsertInto`.
* This updates the test case `InsertInto: overwrite - multiple static partitions - dynamic mode` in `InsertIntoTests`. The result of the SQL command is unchanged, but the SQL command will now use a dynamic overwrite so the test now uses `dynamicOverwriteTest`.

Closes #28993 from rdblue/fix-insert-overwrite-v2-conversion.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-08 16:06:40 -07:00
Kousuke Saruta 371b35d2e0 [SPARK-32214][SQL] The type conversion function generated in makeFromJava for "other" type uses a wrong variable
### What changes were proposed in this pull request?

This PR fixes an inconsistency in `EvaluatePython.makeFromJava`, which creates a type conversion function for some Java/Scala types.

`other` is a type but it should actually pass `obj`:

```scala
case other => (obj: Any) => nullSafeConvert(other)(PartialFunction.empty)
```

This does not change the output because it always returns `null` for unsupported datatypes.

### Why are the changes needed?

To make the codes coherent, and consistent.

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

No.

### How was this patch tested?

No behaviour change.

Closes #29029 from sarutak/fix-makeFromJava.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-08 17:46:25 +09:00
ulysses 65286aec4b [SPARK-30703][SQL][FOLLOWUP] Update SqlBase.g4 invalid comment
### What changes were proposed in this pull request?

Modify the comment of `SqlBase.g4`.

### Why are the changes needed?

`docs/sql-keywords.md` has already moved to `docs/sql-ref-ansi-compliance.md#sql-keywords`.

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

No.

### How was this patch tested?

No need.

Closes #29033 from ulysses-you/SPARK-30703-FOLLOWUP.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-08 11:30:47 +09:00
LantaoJin b5297c43b0 [SPARK-20680][SQL] Spark-sql do not support for creating table with void column datatype
### What changes were proposed in this pull request?

This is the new PR which to address the close one #17953

1. support "void" primitive data type in the `AstBuilder`, point it to `NullType`
2. forbid creating tables with VOID/NULL column type

### Why are the changes needed?

1. Spark is incompatible with hive void type. When Hive table schema contains void type, DESC table will throw an exception in Spark.

>hive> create table bad as select 1 x, null z from dual;
>hive> describe bad;
OK
x	int
z	void

In Spark2.0.x, the behaviour to read this view is normal:
>spark-sql> describe bad;
x       int     NULL
z       void    NULL
Time taken: 4.431 seconds, Fetched 2 row(s)

But in lastest Spark version, it failed with SparkException: Cannot recognize hive type string: void

>spark-sql> describe bad;
17/05/09 03:12:08 ERROR thriftserver.SparkSQLDriver: Failed in [describe bad]
org.apache.spark.SparkException: Cannot recognize hive type string: void
Caused by: org.apache.spark.sql.catalyst.parser.ParseException:
DataType void() is not supported.(line 1, pos 0)
== SQL ==
void
^^^
        ... 61 more
org.apache.spark.SparkException: Cannot recognize hive type string: void

2. Hive CTAS statements throws error when select clause has NULL/VOID type column since HIVE-11217
In Spark, creating table with a VOID/NULL column should throw readable exception message, include

- create data source table (using parquet, json, ...)
- create hive table (with or without stored as)
- CTAS

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

No

### How was this patch tested?

Add unit tests

Closes #28833 from LantaoJin/SPARK-20680_COPY.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-07 18:58:01 -07:00
Ali Smesseim 8b0a54e6ff [SPARK-32057][SQL][TEST-HIVE1.2][TEST-HADOOP2.7] ExecuteStatement: cancel and close should not transiently ERROR
### What changes were proposed in this pull request?
#28671 introduced a change where the order in which CANCELED state for SparkExecuteStatementOperation is set was changed. Before setting the state to CANCELED, `cleanup()` was called which kills the jobs, causing an exception to be thrown inside `execute()`. This causes the state to transiently become ERROR before being set to CANCELED. This PR fixes the order.

### Why are the changes needed?
Bug: wrong operation state is set.

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

### How was this patch tested?
Unit test in SparkExecuteStatementOperationSuite.scala.

Closes #28912 from alismess-db/execute-statement-operation-cleanup-order.

Authored-by: Ali Smesseim <ali.smesseim@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-08 09:28:16 +09:00
Max Gekk 1261fac674 [SPARK-31710][SQL][FOLLOWUP] Allow cast numeric to timestamp by default
### What changes were proposed in this pull request?
1. Set the SQL config `spark.sql.legacy.allowCastNumericToTimestamp` to `true` by default
2. Remove explicit sets of `spark.sql.legacy.allowCastNumericToTimestamp` to `true` in the cast suites.

### Why are the changes needed?
To avoid breaking changes in minor versions (in the upcoming Spark 3.1.0) according to the the semantic versioning guidelines (https://spark.apache.org/versioning-policy.html)

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

### How was this patch tested?
By `CastSuite`.

Closes #29012 from MaxGekk/allow-cast-numeric-to-timestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-07 14:09:40 -07:00
Liang-Chi Hsieh 90b9099064 [SPARK-32163][SQL] Nested pruning should work even with cosmetic variations
### What changes were proposed in this pull request?

This patch proposes to deal with cosmetic variations when processing nested column extractors in `NestedColumnAliasing`. Currently if cosmetic variations are in the nested column extractors, the query is not optimized.

### Why are the changes needed?

If the expressions extracting nested fields have cosmetic variations like qualifier difference, currently nested column pruning cannot work well.

For example, two attributes which are semantically the same, are referred in a query, but the nested column extractors of them are treated differently when we deal with nested column pruning.

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

Yes, fixing a bug in nested column pruning.

### How was this patch tested?

Unit test.

Closes #28988 from viirya/SPARK-32163.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-07 11:17:53 -07:00
fqaiser94@gmail.com 4bbc343a4c [SPARK-31317][SQL] Add withField method to Column
### What changes were proposed in this pull request?

Added a new `withField` method to the `Column` class. This method should allow users to add or replace a `StructField` in a `StructType` column (with very similar semantics to the `withColumn` method on `Dataset`).

### Why are the changes needed?

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

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

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

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

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

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

```

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

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

No.

### How was this patch tested?

New unit tests were added. Jenkins must pass them.

### Related JIRAs:
- https://issues.apache.org/jira/browse/SPARK-22231
- https://issues.apache.org/jira/browse/SPARK-16483

Closes #27066 from fqaiser94/SPARK-22231-withField.

Lead-authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Co-authored-by: fqaiser94 <fqaiser94@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-07 16:34:03 +00:00
Max Gekk 75d342858a [SPARK-32209][SQL] Re-use GetTimestamp in ParseToDate
### What changes were proposed in this pull request?
Replace the combination of expressions `SecondsToTimestamp` and `UnixTimestamp` by `GetTimestamp` in `ParseToDate`.

### Why are the changes needed?
Eliminate unnecessary parsing overhead in: **string -> timestamp -> long (seconds) -> timestamp -> date**. After the changes, the chain will be: **string -> timestamp -> date**.

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

### How was this patch tested?
By existing test suites such as `DateFunctionsSuite`.

Closes #28999 from MaxGekk/ParseToDate-parse-timestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-07 07:26:40 -07:00
ulysses 2e23da2bda [SPARK-31975][SQL] Show AnalysisException when WindowFunction is used without WindowExpression
### What changes were proposed in this pull request?

Add WindowFunction check at `CheckAnalysis`.

### Why are the changes needed?
Provide friendly error msg.

**BEFORE**
```scala
scala> sql("select rank() from values(1)").show
java.lang.UnsupportedOperationException: Cannot generate code for expression: rank()
```

**AFTER**
```scala
scala> sql("select rank() from values(1)").show
org.apache.spark.sql.AnalysisException: Window function rank() requires an OVER clause.;;
Project [rank() AS RANK()#3]
+- LocalRelation [col1#2]
```

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

Yes, user wiill be given a better error msg.

### How was this patch tested?

Pass the newly added UT.

Closes #28808 from ulysses-you/SPARK-31975.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-07 13:39:04 +00:00
Wenchen Fan 5d296ed39e [SPARK-32167][SQL] Fix GetArrayStructFields to respect inner field's nullability together
### What changes were proposed in this pull request?

Fix nullability of `GetArrayStructFields`. It should consider both the original array's `containsNull` and the inner field's nullability.

### Why are the changes needed?

Fix a correctness issue.

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

Yes. See the added test.

### How was this patch tested?

a new UT and end-to-end test

Closes #28992 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-06 20:07:33 -07:00
Kent Yao 59a70879c0 [SPARK-32145][SQL][TEST-HIVE1.2][TEST-HADOOP2.7] ThriftCLIService.GetOperationStatus should include exception's stack trace to the error message
### What changes were proposed in this pull request?

In https://issues.apache.org/jira/browse/SPARK-29283, we only show the error message of root cause to end-users through JDBC client. In some cases, it erases the straightaway messages that we intentionally make to help them for better understanding.

The root cause is somehow obscure for JDBC end-users who only writing SQL queries.

e.g
```
Error running query: org.apache.spark.sql.AnalysisException: The second argument of 'date_sub' function needs to be an integer.;
```

is better than just

```
Caused by: java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2
```

We should do as Hive does in https://issues.apache.org/jira/browse/HIVE-14368

In general, this PR partially reverts SPARK-29283, ports HIVE-14368, and improves test coverage

### Why are the changes needed?

1. Do the same as Hive 2.3 and later for getting an error message in ThriftCLIService.GetOperationStatus
2. The root cause is somehow obscure for JDBC end-users who only writing SQL queries.
3. Consistency with `spark-sql` script

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

Yes, when running queries using thrift server and an error occurs, you will get the full stack traces instead of only the message of the root cause

### How was this patch tested?

add unit test

Closes #28963 from yaooqinn/SPARK-32145.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-06 10:34:31 +00:00
Max Gekk 42f01e314b [SPARK-32130][SQL][FOLLOWUP] Enable timestamps inference in JsonBenchmark
### What changes were proposed in this pull request?
Set the JSON option `inferTimestamp` to `true` for the cases that measure perf of timestamp inference.

### Why are the changes needed?
The PR https://github.com/apache/spark/pull/28966 disabled timestamp inference by default. As a consequence, some benchmarks don't measure perf of timestamp inference from JSON fields. This PR explicitly enable such inference.

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

### How was this patch tested?
By re-generating results of `JsonBenchmark`.

Closes #28981 from MaxGekk/json-inferTimestamps-disable-by-default-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-02 13:26:57 -07:00
TJX2014 0acad589e1 [SPARK-32156][SPARK-31061][TESTS][SQL] Refactor two similar test cases from in HiveExternalCatalogSuite
### What changes were proposed in this pull request?
1.Merge two similar tests for SPARK-31061 and make the code clean.
2.Fix table alter issue due to lose path.

### Why are the changes needed?
Because this two tests for SPARK-31061 is very similar and could be merged.
And the first test case should use `rawTable` instead of `parquetTable` to alter.

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

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

Closes #28980 from TJX2014/master-follow-merge-spark-31061-test-case.

Authored-by: TJX2014 <xiaoxingstack@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-02 10:15:10 -07:00
stczwd f082a7996a [SPARK-31100][SQL] Check namespace existens when setting namespace
## What changes were proposed in this pull request?
Check the namespace existence while calling "use namespace", and throw NoSuchNamespaceException if namespace not exists.

### Why are the changes needed?
Users need to know that the namespace does not exist when they try to set a wrong namespace.

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

### How was this patch tested?
Run all suites and add a test for this

Closes #27900 from stczwd/SPARK-31100.

Authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-02 14:49:40 +00:00
Wenchen Fan f83415629b [MINOR][TEST][SQL] Make in-limit.sql more robust
### What changes were proposed in this pull request?

For queries like `t1d in (SELECT t2d FROM  t2 ORDER  BY t2c LIMIT 2)`, the result can be non-deterministic as the result of the subquery may output different results (it's not sorted by `t2d` and it has shuffle).

This PR makes the test more robust by sorting the output column.

### Why are the changes needed?

avoid flaky test

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

no

### How was this patch tested?

N/A

Closes #28976 from cloud-fan/small.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-02 21:04:26 +09:00
Liang-Chi Hsieh 3f7780d30d [SPARK-32136][SQL] NormalizeFloatingNumbers should work on null struct
### What changes were proposed in this pull request?

This patch fixes wrong groupBy result if the grouping key is a null-value struct.

### Why are the changes needed?

`NormalizeFloatingNumbers` reconstructs a struct if input expression is StructType. If the input struct is null, it will reconstruct a struct with null-value fields, instead of null.

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

Yes, fixing incorrect groupBy result.

### How was this patch tested?

Unit test.

Closes #28962 from viirya/SPARK-32136.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-02 13:56:43 +09:00
Liang-Chi Hsieh 2a52a1b300 [SPARK-32056][SQL][FOLLOW-UP] Coalesce partitions for repartiotion hint and sql when AQE is enabled
### What changes were proposed in this pull request?

As the followup of #28900, this patch extends coalescing partitions to repartitioning using hints and SQL syntax without specifying number of partitions, when AQE is enabled.

### Why are the changes needed?

When repartitionning using hints and SQL syntax, we should follow the shuffling behavior of repartition by expression/range to coalesce partitions when AQE is enabled.

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

Yes. After this change, if users don't specify the number of partitions when repartitioning using `REPARTITION`/`REPARTITION_BY_RANGE` hint or `DISTRIBUTE BY`/`CLUSTER BY`, AQE will coalesce partitions.

### How was this patch tested?

Unit tests.

Closes #28952 from viirya/SPARK-32056-sql.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 16:14:51 -07:00
Max Gekk bcf23307f4 [SPARK-32130][SQL] Disable the JSON option inferTimestamp by default
### What changes were proposed in this pull request?
Set the JSON option `inferTimestamp` to `false` if an user don't pass it as datasource option.

### Why are the changes needed?
To prevent perf regression while inferring schemas from JSON with potential timestamps fields.

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

### How was this patch tested?
- Modified existing tests in `JsonSuite` and `JsonInferSchemaSuite`.
- Regenerated results of `JsonBenchmark` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28966 from MaxGekk/json-inferTimestamps-disable-by-default.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 15:45:39 -07:00
Wenchen Fan 6edb20df83 [SPARK-31935][SQL][FOLLOWUP] Hadoop file system config should be effective in data source options
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28760 to fix the remaining issues:
1. should consider data source options when refreshing cache by path at the end of `InsertIntoHadoopFsRelationCommand`
2. should consider data source options when inferring schema for file source
3. should consider data source options when getting the qualified path in file source v2.

### Why are the changes needed?

We didn't catch these issues in https://github.com/apache/spark/pull/28760, because the test case is to check error when initializing the file system. If we initialize the file system multiple times during a simple read/write action, the test case actually only test the first time.

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

No

### How was this patch tested?

rewrite the test to make sure the entire data source read/write action can succeed.

Closes #28948 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-02 06:09:54 +08:00
Wenchen Fan 7dbd90b68d [SPARK-31797][SQL][FOLLOWUP] TIMESTAMP_SECONDS supports fractional input
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/28534 , to make `TIMESTAMP_SECONDS` function support fractional input as well.

### Why are the changes needed?

Previously the cast function can cast fractional values to timestamp. Now we suggest users to ues these new functions, and we need to cover all the cast use cases.

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

Yes, now `TIMESTAMP_SECONDS` function accepts fractional input.

### How was this patch tested?

new tests

Closes #28956 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 08:03:46 -07:00
angerszhu 15fb5d7677 [SPARK-28169][SQL] Convert scan predicate condition to CNF
### What changes were proposed in this pull request?
Spark can't push down scan predicate condition of **Or**:
Such as if I have a table `default.test`, it's partition col is `dt`,
If we use query :
```
select * from default.test
where dt=20190625 or (dt = 20190626 and id in (1,2,3) )
```

In this case, Spark will resolve **Or** condition as one expression, and since this expr has reference of "id", then it can't been push down.

Base on pr https://github.com/apache/spark/pull/28733, In my PR ,  for SQL like
`select * from default.test`
 `where  dt = 20190626  or  (dt = 20190627  and xxx="a")   `

For this  condition `dt = 20190626  or  (dt = 20190627  and xxx="a"   )`, it will  been converted  to CNF
```
(dt = 20190626 or dt = 20190627) and (dt = 20190626 or xxx = "a" )
```
then condition `dt = 20190626 or dt = 20190627` will be push down when partition pruning

### Why are the changes needed?
Optimize partition pruning

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

### How was this patch tested?
Added UT

Closes #28805 from AngersZhuuuu/cnf-for-partition-pruning.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-01 12:00:15 +00:00
Micah Kornfield a4ba34434c [SPARK-32095][SQL] Update documentation to reflect usage of updated statistics
### What changes were proposed in this pull request?
Update documentation to reflect changes in faf220aad9

I've changed the documentation to reflect updated statistics may be used to improve query plan.

### Why are the changes needed?
I believe the documentation is stale and misleading.

### Does this PR introduce _any_ user-facing change?
Yes, this is a javadoc documentation fix.

### How was this patch tested?
Doc fix.

Closes #28925 from emkornfield/spark-32095.

Authored-by: Micah Kornfield <micahk@google.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-01 07:13:10 +00:00
GuoPhilipse 02f3b80d3a [SPARK-32131][SQL] Fix AnalysisException messages at UNION/EXCEPT/MINUS operations
### What changes were proposed in this pull request?
fix error exception messages during exceptions on Union and set operations

### Why are the changes needed?
Union and set operations can only be performed on tables with the compatible column types,while when we have more than two column, the exception messages will have wrong column index.

Steps to reproduce:

```
drop table if exists test1;
drop table if exists test2;
drop table if exists test3;
create table if not exists test1(id int, age int, name timestamp);
create table if not exists test2(id int, age timestamp, name timestamp);
create table if not exists test3(id int, age int, name int);
insert into test1 select 1,2,'2020-01-01 01:01:01';
insert into test2 select 1,'2020-01-01 01:01:01','2020-01-01 01:01:01';
insert into test3 select 1,3,4;
```

Query1:
```sql
select * from test1 except select * from test2;
```
Result1:
```
Error: org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. timestamp <> int at the second column of the second table;; 'Except false :- Project [id#620, age#621, name#622] : +- SubqueryAlias `default`.`test1` : +- HiveTableRelation `default`.`test1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#620, age#621, name#622] +- Project [id#623, age#624, name#625] +- SubqueryAlias `default`.`test2` +- HiveTableRelation `default`.`test2`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#623, age#624, name#625] (state=,code=0)
```

Query2:

```sql
select * from test1 except select * from test3;
```

Result2:

```
Error: org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types
 int <> timestamp at the 2th column of the second table;
```

the above query1 has the right exception message
the above query2 have the wrong errors information, it may need to change to the following

```
Error: org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types.
int <> timestamp at the  third column of the second table
```

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

### How was this patch tested?
unit test

Closes #28951 from GuoPhilipse/32131-correct-error-messages.

Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-30 23:33:50 -07:00
HyukjinKwon 8194d9ef78 [SPARK-32142][SQL][TESTS] Keep the original tests and codes to avoid potential conflicts in dev
### What changes were proposed in this pull request?

This PR proposes to partially reverts back in the tests and some codes at https://github.com/apache/spark/pull/27728 without touching any behaivours.

Most of changes in tests are back before #27728 by combining `withNestedDataFrame` and `withParquetDataFrame`.

Basically, it addresses the comments https://github.com/apache/spark/pull/27728#discussion_r397655390, and my own comment in another PR at https://github.com/apache/spark/pull/28761#discussion_r446761037

### Why are the changes needed?

For maintenance purpose and to avoid a potential conflicts during backports. And also in case when other codes are matched with this.

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

No, dev-only.

### How was this patch tested?

Manually tested.

Closes #28955 from HyukjinKwon/SPARK-25556-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-01 14:15:02 +09:00
Gabor Somogyi bbd0275dfd [MINOR][SQL] Fix spaces in JDBC connection providers
### What changes were proposed in this pull request?
JDBC connection providers implementation formatted in a wrong way. In this PR I've fixed the formatting.

### Why are the changes needed?
Wrong spacing in JDBC connection providers.

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

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

Closes #28945 from gaborgsomogyi/provider_spacing.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-30 11:18:16 -07:00
Gabor Somogyi 67cb7eaa65 [SPARK-31336][SQL] Support Oracle Kerberos login in JDBC connector
### What changes were proposed in this pull request?
When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.

This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues.

In this PR I've added Oracle support.

What this PR contains:
* Added `OracleConnectionProvider`
* Added `OracleConnectionProviderSuite`

### Why are the changes needed?
Missing JDBC kerberos support.

### Does this PR introduce _any_ user-facing change?
Yes, now user is able to connect to Oracle using kerberos.

### How was this patch tested?
* Additional + existing unit tests
* Test on cluster manually

Closes #28863 from gaborgsomogyi/SPARK-31336.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-30 10:30:22 -07:00
Jungtaek Lim (HeartSaVioR) 5472170a2b [SPARK-29999][SS][FOLLOWUP] Fix test to check the actual metadata log directory
### What changes were proposed in this pull request?

This patch fixes the missed spot - the test initializes FileStreamSinkLog with its "output" directory instead of "metadata" directory, hence the verification against sink log was no-op.

### Why are the changes needed?

Without the fix, the verification against sink log was no-op.

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

No.

### How was this patch tested?

Checked with debugger in test, and verified `allFiles()` returns non-zero entries. (It returned zero entry, as there's no metadata.)

Closes #28930 from HeartSaVioR/SPARK-29999-FOLLOWUP-fix-test.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-30 08:09:18 +00:00
xy_xin 20cd47e82d [SPARK-32030][SQL] Support unlimited MATCHED and NOT MATCHED clauses in MERGE INTO
### What changes were proposed in this pull request?
This PR add unlimited MATCHED and NOT MATCHED clauses in MERGE INTO statement.

### Why are the changes needed?
Now the MERGE INTO syntax is,
```
MERGE INTO [db_name.]target_table [AS target_alias]
 USING [db_name.]source_table [<time_travel_version>] [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> ]
```
It would be nice if we support unlimited MATCHED and NOT MATCHED clauses in MERGE INTO statement, because users may want to deal with different "AND <condition>"s, the result of which just like a series of "CASE WHEN"s. The expected syntax looks like
```
MERGE INTO [db_name.]target_table [AS target_alias]
 USING [db_name.]source_table [<time_travel_version>] [AS source_alias]
 ON <merge_condition>
 [when_matched_clause [, ...]]
 [when_not_matched_clause [, ...]]
```
where when_matched_clause is
```
WHEN MATCHED [ AND <condition> ] THEN <matched_action>
```
and when_not_matched_clause is
```
WHEN NOT MATCHED [ AND <condition> ] THEN <not_matched_action>
 ```
matched_action can be one of
```
DELETE
UPDATE SET * or
UPDATE SET col1 = value1 [, col2 = value2, ...]
```
and not_matched_action can be one of
```
INSERT *
INSERT (col1 [, col2, ...]) VALUES (value1 [, value2, ...])
```
### Does this PR introduce _any_ user-facing change?
Yes. The SQL command changes, but it is backward compatible.

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

Closes #28875 from xianyinxin/SPARK-32030.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-29 13:13:42 +00:00
yi.wu 6fcb70e0ca [SPARK-32055][CORE][SQL] Unify getReader and getReaderForRange in ShuffleManager
### What changes were proposed in this pull request?

This PR tries to unify the method `getReader` and `getReaderForRange` in `ShuffleManager`.

### Why are the changes needed?

Reduce the duplicate codes, simplify the implementation, and for better maintenance.

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #28895 from Ngone51/unify-getreader.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-29 11:37:03 +00:00
Liang-Chi Hsieh 4204a63d4f [SPARK-32056][SQL] Coalesce partitions for repartition by expressions when AQE is enabled
### What changes were proposed in this pull request?

This patch proposes to coalesce partitions for repartition by expressions without specifying number of partitions, when AQE is enabled.

### Why are the changes needed?

When repartition by some partition expressions, users can specify number of partitions or not. If  the number of partitions is specified, we should not coalesce partitions because it breaks user expectation. But if without specifying number of partitions, AQE should be able to coalesce partitions as other shuffling.

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

Yes. After this change, if users don't specify the number of partitions when repartitioning data by expressions, AQE will coalesce partitions.

### How was this patch tested?

Added unit test.

Closes #28900 from viirya/SPARK-32056.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-29 11:33:40 +00:00
Wenchen Fan 835ef425d0 [SPARK-32038][SQL][FOLLOWUP] Make the alias name pretty after float/double normalization
### What changes were proposed in this pull request?

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

This PR proposes to use the name of the original expression, as the alias name of the normalization expression.

### Why are the changes needed?

make the query plan looks pretty when EXPLAIN.

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

No

### How was this patch tested?

manually explain the query

Closes #28919 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:55:19 -07:00
yi.wu 0ec17c989d [SPARK-32090][SQL] Improve UserDefinedType.equal() to make it be symmetrical
### What changes were proposed in this pull request?

This PR fix `UserDefinedType.equal()` by comparing the UDT class instead of checking `acceptsType()`.

### Why are the changes needed?

It's weird that equality comparison between two UDT types can have different result by switching the order:

```scala
// ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass
val udt1 = new ExampleBaseTypeUDT
val udt2 = new ExampleSubTypeUDT
println(udt1 == udt2) // true
println(udt2 == udt1) // false
```

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

Yes.

Before:
```scala
// ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass
val udt1 = new ExampleBaseTypeUDT
val udt2 = new ExampleSubTypeUDT
println(udt1 == udt2) // true
println(udt2 == udt1) // false
```

After:
```scala
// ExampleSubTypeUDT.userClass is a subclass of ExampleBaseTypeUDT.userClass
val udt1 = new ExampleBaseTypeUDT
val udt2 = new ExampleSubTypeUDT
println(udt1 == udt2) // false
println(udt2 == udt1) // false
```

### How was this patch tested?

Added a unit test.

Closes #28923 from Ngone51/fix-udt-equal.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:49:10 -07:00
Yuanjian Li f944603872 [SPARK-32126][SS] Scope Session.active in IncrementalExecution
### What changes were proposed in this pull request?

The `optimizedPlan` in IncrementalExecution should also be scoped in `withActive`.

### Why are the changes needed?

Follow-up of SPARK-30798 for the Streaming side.

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

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

Closes #28936 from xuanyuanking/SPARK-30798-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:35:59 -07:00
Yuanjian Li 6484c14c57 [SPARK-32115][SQL] Fix SUBSTRING to handle integer overflows
### What changes were proposed in this pull request?
Bug fix for overflow case in `UTF8String.substringSQL`.

### Why are the changes needed?
SQL query `SELECT SUBSTRING("abc", -1207959552, -1207959552)` incorrectly returns` "abc"` against expected output of `""`. For query `SUBSTRING("abc", -100, -100)`, we'll get the right output of `""`.

### Does this PR introduce _any_ user-facing change?
Yes, bug fix for the overflow case.

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

Closes #28937 from xuanyuanking/SPARK-32115.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 12:22:44 -07:00
Max Gekk 8c44d74463 [SPARK-32071][SQL][TESTS] Add make_interval benchmark
### What changes were proposed in this pull request?
Add benchmarks for interval constructor `make_interval` and measure perf of 4 cases:
1. Constant (year, month)
2. Constant (week, day)
3. Constant (hour, minute, second, second fraction)
4. All fields are NOT constant.

The benchmark results are generated 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

### Why are the changes needed?
To have a base line for future perf improvements of `make_interval`, and to prevent perf regressions in the future.

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

### How was this patch tested?
By running `IntervalBenchmark` via:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.IntervalBenchmark"
```

Closes #28905 from MaxGekk/benchmark-make_interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-27 17:54:06 -07:00
GuoPhilipse ac3a0551d8 [SPARK-32088][PYTHON] Pin the timezone in timestamp_seconds doctest
### What changes were proposed in this pull request?

Add American timezone during timestamp_seconds doctest

### Why are the changes needed?

`timestamp_seconds` doctest in `functions.py` used default timezone to get expected result
For example:

```python
>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect()
[Row(ts=datetime.datetime(2008, 12, 25, 7, 30))]
```

But when we have a non-american timezone, the test case will get different test result.

For example, when we set current timezone as `Asia/Shanghai`, the test result will be

```
[Row(ts=datetime.datetime(2008, 12, 25, 23, 30))]
```

So no matter where we run the test case ,we will always get the expected permanent result if we set the timezone on one specific area.

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

No

### How was this patch tested?

Unit test

Closes #28932 from GuoPhilipse/SPARK-32088-fix-timezone-issue.

Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-26 19:06:31 -07:00
Pablo Langa bbb2cba615 [SPARK-32025][SQL] Csv schema inference problems with different types in the same column
### What changes were proposed in this pull request?

This pull request fixes a bug present in the csv type inference.
We have problems when we have different types in the same column.

**Previously:**
```
$ cat /example/f1.csv
col1
43200000
true

spark.read.csv(path="file:///example/*.csv", header=True, inferSchema=True).show()
+----+
|col1|
+----+
|null|
|true|
+----+

root
 |-- col1: boolean (nullable = true)
```
**Now**
```
spark.read.csv(path="file:///example/*.csv", header=True, inferSchema=True).show()
+-------------+
|col1          |
+-------------+
|43200000 |
|true           |
+-------------+

root
 |-- col1: string (nullable = true)
```

Previously the hierarchy of type inference is the following:

> IntegerType
> > LongType
> > > DecimalType
> > > > DoubleType
> > > > > TimestampType
> > > > > > BooleanType
> > > > > > > StringType

So, when, for example, we have integers in one column, and the last element is a boolean, all the column is inferred as a boolean column incorrectly and all the number are shown as null when you see the data

We need the following hierarchy. When we have different numeric types in the column it will be resolved correctly. And when we have other different types it will be resolved as a String type column
> IntegerType
> > LongType
> > > DecimalType
> > > > DoubleType
> > > > > StringType

> TimestampType
> > StringType

> BooleanType
> > StringType

> StringType

### Why are the changes needed?

Fix the bug explained

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

No

### How was this patch tested?

Unit test and manual tests

Closes #28896 from planga82/feature/SPARK-32025_csv_inference.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-26 10:41:27 +09:00
yi.wu 47fb9d6054 [SPARK-32087][SQL] Allow UserDefinedType to use encoder to deserialize rows in ScalaUDF as well
### What changes were proposed in this pull request?

This PR tries to address the comment: https://github.com/apache/spark/pull/28645#discussion_r442183888
It changes `canUpCast/canCast` to allow cast from sub UDT to base UDT, in order to achieve the goal to allow UserDefinedType to use `ExpressionEncoder` to deserialize rows in ScalaUDF as well.

One thing that needs to mention is, even we allow cast from sub UDT to base UDT, it doesn't really do the cast in `Cast`. Because, yet, sub UDT and base UDT are considered as the same type(because of #16660), see:

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala (L81-L86)

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala (L92-L95)

Therefore, the optimize rule `SimplifyCast` will eliminate the cast at the end.

### Why are the changes needed?

Reduce the special case caused by `UserDefinedType` in `ResolveEncodersInUDF` and `ScalaUDF`.

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

No.

### How was this patch tested?

It should be covered by the test of `SPARK-19311`, which is also updated a little in this PR.

Closes #28920 from Ngone51/fix-udf-udt.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 14:50:45 +00:00
Bryan Cutler df04107934 [SPARK-32080][SPARK-31998][SQL] Simplify ArrowColumnVector ListArray accessor
### What changes were proposed in this pull request?

This change simplifies the ArrowColumnVector ListArray accessor to use provided Arrow APIs available in v0.15.0 to calculate element indices.

### Why are the changes needed?

This simplifies the code by avoiding manual calculations on the Arrow offset buffer and makes use of more stable APIs.

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

No

### How was this patch tested?

Existing tests

Closes #28915 from BryanCutler/arrow-simplify-ArrowColumnVector-ListArray-SPARK-32080.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-24 22:13:54 +09:00
ulysses 9f540fac2e [SPARK-32062][SQL] Reset listenerRegistered in SparkSession
### What changes were proposed in this pull request?

Reset listenerRegistered when application end.

### Why are the changes needed?

Within a jvm, stop and create `SparkContext` multi times will cause the bug.

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

No.

### How was this patch tested?

Add UT.

Closes #28899 from ulysses-you/SPARK-32062.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 04:50:46 +00:00
Max Gekk 045106e29d [SPARK-32072][CORE][TESTS] Fix table formatting with benchmark results
### What changes were proposed in this pull request?
Set column width w/ benchmark names to maximum of either
1. 40 (before this PR) or
2. The length of benchmark name or
3. Maximum length of cases names

### Why are the changes needed?
To improve readability of benchmark results. For example, `MakeDateTimeBenchmark`.

Before:
```
make_timestamp():                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
prepare make_timestamp()                           3636           3673          38          0.3        3635.7       1.0X
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             94             99           4         10.7          93.8      38.8X
make_timestamp(2019, 1, 2, 3, 4, 60.000000)             68             80          13         14.6          68.3      53.2X
make_timestamp(2019, 12, 31, 23, 59, 60.00)             65             79          19         15.3          65.3      55.7X
make_timestamp(*, *, *, 3, 4, 50.123456)            271            280          14          3.7         270.7      13.4X
```

After:
```
make_timestamp():                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------
prepare make_timestamp()                              3694           3745          82          0.3        3694.0       1.0X
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             82             90           9         12.2          82.3      44.9X
make_timestamp(2019, 1, 2, 3, 4, 60.000000)             72             77           5         13.9          71.9      51.4X
make_timestamp(2019, 12, 31, 23, 59, 60.00)             67             71           5         15.0          66.8      55.3X
make_timestamp(*, *, *, 3, 4, 50.123456)               273            289          14          3.7         273.2      13.5X
```

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

### How was this patch tested?
By re-generating benchmark results for `MakeDateTimeBenchmark`:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark"
```
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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28906 from MaxGekk/benchmark-table-formatting.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 04:43:53 +00:00
Max Gekk e00f43cb86 [SPARK-32043][SQL] Replace Decimal by Int op in make_interval and make_timestamp
### What changes were proposed in this pull request?
Replace Decimal by Int op in the `MakeInterval` & `MakeTimestamp` expression. For instance, `(secs * Decimal(MICROS_PER_SECOND)).toLong` can be replaced by the unscaled long because the former one already contains microseconds.

### Why are the changes needed?
To improve performance.

Before:
```
make_timestamp():                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
...
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             94             99           4         10.7          93.8      38.8X
```

After:
```
make_timestamp(2019, 1, 2, 3, 4, 50.123456)             76             92          15         13.1          76.5      48.1X
```

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

### How was this patch tested?
- By existing test suites `IntervalExpressionsSuite`, `DateExpressionsSuite` and etc.
- Re-generate results of `MakeDateTimeBenchmark` 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_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28886 from MaxGekk/make_interval-opt-decimal.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-23 11:45:12 +00:00
Max Gekk fcf9768098 [SPARK-32052][SQL] Extract common code from date-time field expressions
### What changes were proposed in this pull request?
Extract common code from the expressions that get date or time fields from input dates/timestamps to new expressions `GetDateField` and `GetTimeField`, and re-use the common traits from the affected classes.

### Why are the changes needed?
Code deduplication improves maintainability.

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

### How was this patch tested?
By `DateExpressionsSuite`

Closes #28894 from MaxGekk/get-date-time-field-expr.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-23 06:13:55 +00:00