Commit graph

25771 commits

Author SHA1 Message Date
Takeshi Yamamuro 0032d85153 [SPARK-29968][SQL] Remove the Predicate code from SparkPlan
### What changes were proposed in this pull request?

This is to refactor Predicate code; it mainly removed `newPredicate` from `SparkPlan`.
Modifications are listed below;
 - Move `Predicate` from `o.a.s.sqlcatalyst.expressions.codegen.GeneratePredicate.scala` to `o.a.s.sqlcatalyst.expressions.predicates.scala`
 - To resolve the name conflict,  rename `o.a.s.sqlcatalyst.expressions.codegen.Predicate` to `o.a.s.sqlcatalyst.expressions.BasePredicate`
 - Extend `CodeGeneratorWithInterpretedFallback ` for `BasePredicate`

This comes from the cloud-fan suggestion: https://github.com/apache/spark/pull/26420#discussion_r348005497

### Why are the changes needed?

For better code/test coverage.

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

No.

### How was this patch tested?

Existing tests.

Closes #26604 from maropu/RefactorPredicate.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-20 21:13:51 +08:00
Nikita Konda 5a70af7a6c [SPARK-29029][SQL] Use AttributeMap in PhysicalOperation.collectProjectsAndFilters
### What changes were proposed in this pull request?

This PR fixes the issue of substituting aliases while collecting filters in  `PhysicalOperation.collectProjectsAndFilters`. When the `AttributeReference` in alias map differs from the `AttributeReference` in filter condition only in qualifier, it does not substitute alias and throws exception saying `key videoid#47L not found` in the following scenario.

```
[1] Project [userid#0]
+- [2] Filter (isnotnull(videoid#47L) && NOT (videoid#47L = 30))
   +- [3] Project [factorial(videoid#1) AS videoid#47L, userid#0]
      +- [4] Filter (isnotnull(avebitrate#2) && (avebitrate#2 < 10))
         +- [5] Relation[userid#0,videoid#1,avebitrate#2]
```

### Why are the changes needed?

We need to use `AttributeMap` where the key is `AttributeReference`'s `ExprId` instead of `Map[Attribute, Expression]` while collecting and substituting aliases in `PhysicalOperation.collectProjectsAndFilters`.

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

### How was this patch tested?
New unit tests were added in `TestPhysicalOperation` which reproduces the bug

Closes #25761 from nikitagkonda/SPARK-29029-use-attributemap-for-aliasmap-in-physicaloperation.

Authored-by: Nikita Konda <nikita.konda@workday.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-19 20:01:42 -08:00
Wenchen Fan 9e58b10c8e [SPARK-29945][SQL] do not handle negative sign specially in the parser
### What changes were proposed in this pull request?

Remove the special handling of the negative sign in the parser (interval literal and type constructor)

### Why are the changes needed?

The negative sign is an operator (UnaryMinus). We don't need to handle it specially, which is kind of doing constant folding at parser side.

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

The error message becomes a little different. Now it reports type mismatch for the `-` operator.

### How was this patch tested?

existing tests

Closes #26578 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-11-20 11:08:04 +09:00
Maxim Gekk 40b8a08b8b [SPARK-29963][SQL][TESTS] Check formatting timestamps up to microsecond precision by JSON/CSV datasource
### What changes were proposed in this pull request?
In the PR, I propose to add tests from the commit 47cb1f359a for Spark 2.4 that check formatting of timestamp strings for various seconds fractions.

### Why are the changes needed?
To make sure that current behavior is the same as in Spark 2.4

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

### How was this patch tested?
By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`.

Closes #26601 from MaxGekk/format-timestamp-micros-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-20 10:34:25 +09:00
Liang-Chi Hsieh e753aa30e6 [SPARK-29964][BUILD] lintr github workflows failed due to buggy GnuPG
### What changes were proposed in this pull request?

Linter (R) github workflows failed sometimes like:

https://github.com/apache/spark/pull/26509/checks?check_run_id=310718016

Failed message:
```
Executing: /tmp/apt-key-gpghome.8r74rQNEjj/gpg.1.sh --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9
gpg: connecting dirmngr at '/tmp/apt-key-gpghome.8r74rQNEjj/S.dirmngr' failed: IPC connect call failed
gpg: keyserver receive failed: No dirmngr
##[error]Process completed with exit code 2.
```

It is due to a buggy GnuPG. Context:
https://github.com/sbt/website/pull/825
https://github.com/sbt/sbt/issues/4261
https://github.com/microsoft/WSL/issues/3286

### Why are the changes needed?

Make lint-r github workflows work.

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

No

### How was this patch tested?

Pass github workflows.

Closes #26602 from viirya/SPARK-29964.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-19 15:56:50 -08:00
John Bauer e804ed5e33 [SPARK-29691][ML][PYTHON] ensure Param objects are valid in fit, transform
modify Param._copyValues to check valid Param objects supplied as extra

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

Estimator.fit() and Model.transform() accept a dictionary of extra parameters whose values are used to overwrite those supplied at initialization or by default.  Additionally, the ParamGridBuilder.addGrid accepts a parameter and list of values. The keys are presumed to be valid Param objects. This change adds a check that only Param objects are supplied as keys.

### Why are the changes needed?

Param objects are created by and bound to an instance of Params (Estimator, Model, or Transformer). They may be obtained from their parent as attributes, or by name through getParam.

The documentation does not state that keys must be valid Param objects, nor describe how one may be obtained. The current behavior is to silently ignore keys which are not valid Param objects.

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

If the user does not pass in a Param object as required for keys in `extra` for Estimator.fit() and Model.transform(), and `param` for ParamGridBuilder.addGrid, an error will be raised indicating it is an invalid object.

### How was this patch tested?

Added method test_copy_param_extras_check to test_param.py.   Tested with Python 3.7

Closes #26527 from JohnHBauer/paramExtra.

Authored-by: John Bauer <john.h.bauer@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-11-19 14:15:00 -08:00
Wenchen Fan 3d2a6f464f [SPARK-29906][SQL] AQE should not introduce extra shuffle for outermost limit
### What changes were proposed in this pull request?

`AdaptiveSparkPlanExec` should forward `executeCollect` and `executeTake` to the underlying physical plan.

### Why are the changes needed?

some physical plan has optimization in `executeCollect` and `executeTake`. For example, `CollectLimitExec` won't do shuffle for outermost limit.

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

no

### How was this patch tested?

a new test

This closes #26560

Closes #26576 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-11-19 10:39:38 -08:00
Jobit Mathew 6fb8b86065 [SPARK-29913][SQL] Improve Exception in postgreCastToBoolean
### What changes were proposed in this pull request?
Exception improvement.

### Why are the changes needed?
After selecting pgSQL dialect, queries which are failing because of wrong syntax will give long exception stack trace. For example,
`explain select cast ("abc" as boolean);`

Current output:

> ERROR SparkSQLDriver: Failed in [explain select cast ("abc" as boolean)]
> java.lang.IllegalArgumentException: invalid input syntax for type boolean: abc
> 	at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$2(PostgreCastToBoolean.scala:51)
> 	at org.apache.spark.sql.catalyst.expressions.CastBase.buildCast(Cast.scala:277)
> 	at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$1(PostgreCastToBoolean.scala:44)
> 	at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeEval(Cast.scala:773)
> 	at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:460)
> 	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
> 	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:286)
> 	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:286)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:291)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
> 	at org.apache.spark.sql.catalyst.plans.QueryPlan.
>       .
>       .
>       .

### Does this PR introduce any user-facing change?
Yes. After this PR, output for above query will be:

> == Physical Plan ==
> org.apache.spark.sql.AnalysisException: invalid input syntax for type boolean: abc;
>
> Time taken: 0.044 seconds, Fetched 1 row(s)
> 19/11/15 15:38:57 INFO SparkSQLCLIDriver: Time taken: 0.044 seconds, Fetched 1 row(s)

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

Closes #26546 from jobitmathew/pgsqlexception.

Authored-by: Jobit Mathew <jobit.mathew@huawei.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 21:30:38 +08:00
Kent Yao 79ed4ae2db [SPARK-29926][SQL] Fix weird interval string whose value is only a dangling decimal point
### What changes were proposed in this pull request?

Currently, we support to parse '1. second' to 1s or even '. second' to 0s.

```sql
-- !query 118
select interval '1. seconds'
-- !query 118 schema
struct<1 seconds:interval>
-- !query 118 output
1 seconds

-- !query 119
select interval '. seconds'
-- !query 119 schema
struct<0 seconds:interval>
-- !query 119 output
0 seconds
```

```sql
postgres=# select interval '1. second';
ERROR:  invalid input syntax for type interval: "1. second"
LINE 1: select interval '1. second';

postgres=# select interval '. second';
ERROR:  invalid input syntax for type interval: ". second"
LINE 1: select interval '. second';
```
We fix this by fixing the new interval parser's VALUE_FRACTIONAL_PART state

With further digging, we found that 1. is valid in python, r, scala, and presto and so on... so this PR
ONLY forbid the invalid interval value in the form of  '. seconds'.

### Why are the changes needed?

bug fix

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

yes, now we treat '. second' .... as invalid intervals
### How was this patch tested?

add ut

Closes #26573 from yaooqinn/SPARK-29926.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 21:01:26 +08:00
jiake a8d98833b8 [SPARK-29893] improve the local shuffle reader performance by changing the reading task number from 1 to multi
### What changes were proposed in this pull request?
This PR update the local reader task number from 1 to multi `partitionStartIndices.length`.

### Why are the changes needed?
Improve the performance of local shuffle reader.

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

### How was this patch tested?
Existing UTs

Closes #26516 from JkSelf/improveLocalShuffleReader.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 19:18:08 +08:00
wangguangxin.cn ffc9753037 [SPARK-29918][SQL] RecordBinaryComparator should check endianness when compared by long
### What changes were proposed in this pull request?
This PR try to make sure the comparison results of  `compared by 8 bytes at a time` and `compared by bytes wise` in RecordBinaryComparator is *consistent*, by reverse long bytes if it is little-endian and using Long.compareUnsigned.

### Why are the changes needed?
If the architecture supports unaligned or the offset is 8 bytes aligned, `RecordBinaryComparator` compare 8 bytes at a time by reading 8 bytes as a long.  Related code is
```
    if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) {
      while (i <= leftLen - 8) {
        final long v1 = Platform.getLong(leftObj, leftOff + i);
        final long v2 = Platform.getLong(rightObj, rightOff + i);
        if (v1 != v2) {
          return v1 > v2 ? 1 : -1;
        }
        i += 8;
      }
    }
```

Otherwise, it will compare bytes by bytes.  Related code is
```
    while (i < leftLen) {
      final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff;
      final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff;
      if (v1 != v2) {
        return v1 > v2 ? 1 : -1;
      }
      i += 1;
    }
```

However, on little-endian machine,  the result of *compared by a long value* and *compared bytes by bytes* maybe different.

For two same records, its offsets may vary in the first run and second run, which will lead to compare them using long comparison or byte-by-byte comparison, the result maybe different.

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

### How was this patch tested?
Add new test cases in RecordBinaryComparatorSuite

Closes #26548 from WangGuangxin/binary_comparator.

Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 16:10:22 +08:00
Wenchen Fan 16134d6d0f [SPARK-29948][SQL] make the default alias consistent between date, timestamp and interval
### What changes were proposed in this pull request?

Update `Literal.sql` to make date, timestamp and interval consistent. They should all use the `TYPE 'value'` format.

### Why are the changes needed?

Make the default alias consistent. For example, without this patch we will see
```
scala> sql("select interval '1 day', date '2000-10-10'").show
+------+-----------------+
|1 days|DATE '2000-10-10'|
+------+-----------------+
|1 days|       2000-10-10|
+------+-----------------+
```

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

no

### How was this patch tested?

existing tests

Closes #26579 from cloud-fan/sql.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 15:37:35 +08:00
LantaoJin 5ac37a8265 [SPARK-29869][SQL] improve error message in HiveMetastoreCatalog#convertToLogicalRelation
### What changes were proposed in this pull request?
In our production, HiveMetastoreCatalog#convertToLogicalRelation throws AssertError occasionally:
```sql
scala> spark.table("hive_table").show
java.lang.AssertionError: assertion failed
  at scala.Predef$.assert(Predef.scala:208)
  at org.apache.spark.sql.hive.HiveMetastoreCatalog.convertToLogicalRelation(HiveMetastoreCatalog.scala:261)
  at org.apache.spark.sql.hive.HiveMetastoreCatalog.convert(HiveMetastoreCatalog.scala:137)
  at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:220)
  at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:207)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$2(AnalysisHelper.scala:108)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:108)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$4(AnalysisHelper.scala:113)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:113)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:73)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:72)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:29)
  at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:207)
  at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:191)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:130)
  at scala.collection.IndexedSeqOptimized.foldLeft(IndexedSeqOptimized.scala:60)
  at scala.collection.IndexedSeqOptimized.foldLeft$(IndexedSeqOptimized.scala:68)
  at scala.collection.mutable.ArrayBuffer.foldLeft(ArrayBuffer.scala:49)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:127)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:119)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:119)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:168)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:162)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:122)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:98)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:98)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:146)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:201)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:145)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:66)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:63)
  at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:63)
  at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:55)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:86)
  at org.apache.spark.sql.SparkSession.table(SparkSession.scala:585)
  at org.apache.spark.sql.SparkSession.table(SparkSession.scala:581)
  ... 47 elided
````
Most of cases occurred in reading a table which created by an old Spark version.
After recreated the table, the issue will be gone.

After deep dive, the root cause is this external table is a non-partitioned table but the `LOCATION` set to a partitioned path {{/tablename/dt=yyyymmdd}}. The partitionSpec is inferred.

### Why are the changes needed?
Above error message is very confused. We need more details about assert failure information.

This issue caused by `PartitioningAwareFileIndex#inferPartitioning()`. For non-HiveMetastore Spark, it's useful. But for Hive table, it shouldn't infer partition if Hive tell us it's a non partitioned table. (new added)

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

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

Closes #26499 from LantaoJin/SPARK-29869.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 15:22:08 +08:00
yudovin 2e71a6e7ba [SPARK-27558][CORE] Gracefully cleanup task when it fails with OOM exception
### What changes were proposed in this pull request?

When a task fails with OOM exception, the `UnsafeInMemorySorter.array` could be `null`. In the meanwhile, the `cleanupResources()` on task completion would call `UnsafeInMemorySorter.getMemoryUsage` in turn, and that lead to another NPE thrown.

### Why are the changes needed?

Check if `array` is null in `UnsafeInMemorySorter.getMemoryUsage` and it should help to avoid NPE.

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

### How was this patch tested?
It was tested manually.

Closes #26349 from ayudovin/fix-npe-in-listener.

Authored-by: yudovin <artsiom.yudovin@profitero.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-11-18 22:05:34 -08:00
Terry Kim 3d45779b68 [SPARK-29728][SQL] Datasource V2: Support ALTER TABLE RENAME TO
### What changes were proposed in this pull request?

This PR adds `ALTER TABLE a.b.c RENAME TO x.y.x` support for V2 catalogs.

### Why are the changes needed?

The current implementation doesn't support this command V2 catalogs.

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

Yes, now the renaming table works for v2 catalogs:
```
scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show
+---------+---------+
|namespace|tableName|
+---------+---------+
|  ns1.ns2|      old|
+---------+---------+

scala> spark.sql("ALTER TABLE testcat.ns1.ns2.old RENAME TO testcat.ns1.ns2.new").show

scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show
+---------+---------+
|namespace|tableName|
+---------+---------+
|  ns1.ns2|      new|
+---------+---------+
```
### How was this patch tested?

Added unit tests.

Closes #26539 from imback82/rename_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 12:03:29 +08:00
shivsood a834dba120 Revert "[SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils
This reverts commit f7e53865 i.e PR #26301 from master

Closes #26583 from shivsood/revert_29644_master.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-18 18:44:16 -08:00
Yuming Wang 28a502c6e9 [SPARK-28527][FOLLOW-UP][SQL][TEST] Add guides for ThriftServerQueryTestSuite
### What changes were proposed in this pull request?
This PR add guides for `ThriftServerQueryTestSuite`.

### Why are the changes needed?
Add guides

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

### How was this patch tested?
N/A

Closes #26587 from wangyum/SPARK-28527-FOLLOW-UP.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-18 18:13:11 -08:00
HyukjinKwon 882f54b0a3 [SPARK-29870][SQL][FOLLOW-UP] Keep CalendarInterval's toString
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/26418. This PR removed `CalendarInterval`'s `toString` with an unfinished changes.

### Why are the changes needed?

1. Ideally we should make each PR isolated and separate targeting one issue without touching unrelated codes.

2. There are some other places where the string formats were exposed to users. For example:

    ```scala
    scala> sql("select interval 1 days as a").selectExpr("to_csv(struct(a))").show()
    ```
    ```
    +--------------------------+
    |to_csv(named_struct(a, a))|
    +--------------------------+
    |      "CalendarInterval...|
    +--------------------------+
    ```

3.  Such fixes:

    ```diff
     private def writeMapData(
        map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = {
      val keyArray = map.keyArray()
    + val keyString = mapType.keyType match {
    +   case CalendarIntervalType =>
    +    (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i))
    +   case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString
    + }
    ```

    can cause performance regression due to type dispatch for each map.

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

Yes, see 2. case above.

### How was this patch tested?

Manually tested.

Closes #26572 from HyukjinKwon/SPARK-29783.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-19 09:11:41 +09:00
HyukjinKwon 8469614c05 [SPARK-25694][SQL][FOLLOW-UP] Move 'spark.sql.defaultUrlStreamHandlerFactory.enabled' into StaticSQLConf.scala
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency.

### Why are the changes needed?

To put the similar configurations together and for readability.

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

No.

### How was this patch tested?

Manually tested as described in https://github.com/apache/spark/pull/26530.

Closes #26570 from HyukjinKwon/SPARK-25694.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-19 09:08:20 +09:00
Hossein 9514b822a7 [SPARK-29777][SPARKR] SparkR::cleanClosure aggressively removes a function required by user function
### What changes were proposed in this pull request?
The implementation for walking through the user function AST and picking referenced variables and functions, had an optimization to skip a branch if it had already seen it. This runs into an interesting problem in the following example

```
df <- createDataFrame(data.frame(x=1))
f1 <- function(x) x + 1
f2 <- function(x) f1(x) + 2
dapplyCollect(df, function(x) { f1(x); f2(x) })
```
Results in error:
```
org.apache.spark.SparkException: R computation failed with
 Error in f1(x) : could not find function "f1"
Calls: compute -> computeFunc -> f2
```

### Why are the changes needed?
Bug fix

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

### How was this patch tested?
Unit tests in `test_utils.R`

Closes #26429 from falaki/SPARK-29777.

Authored-by: Hossein <hossein@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-19 09:04:59 +09:00
Kent Yao ea010a2bc2 [SPARK-29873][SQL][TEST][FOLLOWUP] set operations should not escape when regen golden file with --SET --import both specified
### What changes were proposed in this pull request?

When regenerating golden files, the set operations via `--SET` will not be done, but those with --import should be exceptions because we need the set command.

### Why are the changes needed?

fix test tool.
### Does this PR introduce any user-facing change?

### How was this patch tested?

add ut, but I'm not sure we need these tests for tests itself.
cc maropu cloud-fan

Closes #26557 from yaooqinn/SPARK-29873.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 01:32:13 +08:00
Kent Yao ae6b711b26 [SPARK-29941][SQL] Add ansi type aliases for char and decimal
### What changes were proposed in this pull request?

Checked with SQL Standard and PostgreSQL

> CHAR is equivalent to CHARACTER. DEC is equivalent to DECIMAL. INT is equivalent to INTEGER. VARCHAR is equivalent to CHARACTER VARYING. ...

```sql
postgres=# select dec '1.0';
numeric
---------
1.0
(1 row)

postgres=# select CHARACTER '. second';
  bpchar
----------
 . second
(1 row)

postgres=# select CHAR '. second';
  bpchar
----------
 . second
(1 row)
```

### Why are the changes needed?

For better ansi support
### Does this PR introduce any user-facing change?

yes, we add character as char and dec as decimal

### How was this patch tested?

add ut

Closes #26574 from yaooqinn/SPARK-29941.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 23:30:31 +08:00
fuwhu c32e228689 [SPARK-29859][SQL] ALTER DATABASE (SET LOCATION) should look up catalog like v2 commands
### What changes were proposed in this pull request?
Add AlterNamespaceSetLocationStatement, AlterNamespaceSetLocation, AlterNamespaceSetLocationExec to make ALTER DATABASE (SET LOCATION) look up catalog like v2 commands.
And also refine the code of AlterNamespaceSetProperties, AlterNamespaceSetPropertiesExec, DescribeNamespace, DescribeNamespaceExec to use SupportsNamespaces instead of CatalogPlugin for catalog parameter.

### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes, add "ALTER NAMESPACE ... SET LOCATION" whose function is same as "ALTER DATABASE ... SET LOCATION" and "ALTER SCHEMA ... SET LOCATION".

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

Closes #26562 from fuwhu/SPARK-29859.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 20:40:23 +08:00
Kent Yao 50f6d930da [SPARK-29870][SQL] Unify the logic of multi-units interval string to CalendarInterval
### What changes were proposed in this pull request?

We now have two different implementation for multi-units interval strings to CalendarInterval type values.

One is used to covert interval string literals to CalendarInterval. This approach will re-delegate the interval string to spark parser which handles the string as a `singleInterval` -> `multiUnitsInterval` -> eventually call `IntervalUtils.fromUnitStrings`

The other is used in `Cast`, which eventually calls `IntervalUtils.stringToInterval`. This approach is ~10 times faster than the other.

We should unify these two for better performance and simple logic. this pr uses the 2nd approach.

### Why are the changes needed?

We should unify these two for better performance and simple logic.

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

no

### How was this patch tested?

we shall not fail on existing uts

Closes #26491 from yaooqinn/SPARK-29870.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 15:50:06 +08:00
Kent Yao 5cebe587c7 [SPARK-29783][SQL] Support SQL Standard/ISO_8601 output style for interval type
### What changes were proposed in this pull request?

Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table.

Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval
-- | -- | -- | -- | --
Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds
SQL STANDARD  | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06
ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S

### Why are the changes needed?

for ANSI SQL support
### Does this PR introduce any user-facing change?

yes,interval out now has 3 output styles
### How was this patch tested?

add new unit tests

cc cloud-fan maropu MaxGekk HyukjinKwon thanks.

Closes #26418 from yaooqinn/SPARK-29783.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 15:42:22 +08:00
gschiavon 73912379d0 [SPARK-29020][SQL] Improving array_sort behaviour
### What changes were proposed in this pull request?
I've noticed that there are two functions to sort arrays sort_array and array_sort.

sort_array is from 1.5.0 and it has the possibility of ordering both ascending and descending

array_sort is from 2.4.0 and it only has the possibility of ordering in ascending.

Basically I just added the possibility of ordering either ascending or descending using array_sort.

I think it would be good to have unified behaviours and not having to user sort_array when you want to order in descending order.
Imagine that you are new to spark, I'd like to be able to sort array using the newest spark functions.

### Why are the changes needed?
Basically to be able to sort the array in descending order using *array_sort* instead of using *sort_array* from 1.5.0

### Does this PR introduce any user-facing change?
Yes, now you are able to sort the array in descending order. Note that it has the same behaviour with nulls than sort_array

### How was this patch tested?
Test's added

This is the link to the [jira](https://issues.apache.org/jira/browse/SPARK-29020)

Closes #25728 from Gschiavon/improving-array-sort.

Lead-authored-by: gschiavon <german.schiavon@lifullconnect.com>
Co-authored-by: Takuya UESHIN <ueshin@databricks.com>
Co-authored-by: gschiavon <Gschiavon@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-18 16:07:05 +09:00
Zhou Jiang ee3bd6d768 [SPARK-25694][SQL] Add a config for URL.setURLStreamHandlerFactory
### What changes were proposed in this pull request?

Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory`

### Why are the changes needed?

This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](https://github.com/apache/spark/pull/17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users.

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

Yes. This provides a new user-configurable property.
By default, the behavior is unchanged.

### How was this patch tested?

Manual testing.

**BEFORE**
```
$ build/sbt package
$ bin/spark-shell
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
java.lang.Error: factory already defined
  at java.net.URL.setURLStreamHandlerFactory(URL.java:1134)
  ... 47 elided
```

**AFTER**
```
$ build/sbt package
$ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
```

Closes #26530 from jiangzho/master.

Lead-authored-by: Zhou Jiang <zhou_jiang@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: zhou-jiang <zhou_jiang@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-11-18 05:44:00 +00:00
Dongjoon Hyun 42f8f79ff0 [SPARK-29936][R] Fix SparkR lint errors and add lint-r GitHub Action
### What changes were proposed in this pull request?

This PR fixes SparkR lint errors and adds `lint-r` GitHub Action to protect the branch.

### Why are the changes needed?

It turns out that we currently don't run it. It's recovered yesterday. However, after that, our Jenkins linter jobs (`master`/`branch-2.4`) has been broken on `lint-r` tasks.

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

No.

### How was this patch tested?

Pass the GitHub Action on this PR in addition to Jenkins R and AppVeyor R.

Closes #26564 from dongjoon-hyun/SPARK-29936.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-17 21:01:01 -08:00
HyukjinKwon f280c6aa54 [SPARK-29378][R][FOLLOW-UP] Remove manual installation of Arrow dependencies in AppVeyor build
### What changes were proposed in this pull request?

This PR remove manual installation of Arrow dependencies in AppVeyor build

### Why are the changes needed?

It's unnecessary. See https://github.com/apache/spark/pull/26555#discussion_r347178368

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

No

### How was this patch tested?

AppVeyor will test.

Closes #26566 from HyukjinKwon/SPARK-29378.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-18 12:54:21 +09:00
xy_xin d83cacfcf5 [SPARK-29907][SQL] Move DELETE/UPDATE/MERGE relative rules to dmlStatementNoWith to support cte
### What changes were proposed in this pull request?

SPARK-27444 introduced `dmlStatementNoWith` so that any dml that needs cte support can leverage it. It be better if we move DELETE/UPDATE/MERGE rules to `dmlStatementNoWith`.

### Why are the changes needed?
Wit this change, we can support syntax like "With t AS (SELECT) DELETE FROM xxx", and so as UPDATE/MERGE.

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

### How was this patch tested?

New cases added.

Closes #26536 from xianyinxin/SPARK-29907.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 11:48:56 +08:00
zhengruifeng c5f644c6eb [SPARK-16872][ML][PYSPARK] Impl Gaussian Naive Bayes Classifier
### What changes were proposed in this pull request?
support `modelType` `gaussian`

### Why are the changes needed?
current modelTypes do not support continuous data

### Does this PR introduce any user-facing change?
yes, add a `modelType` option

### How was this patch tested?
existing testsuites and added ones

Closes #26413 from zhengruifeng/gnb.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2019-11-18 10:05:42 +08:00
Maxim Gekk 5eb8973f87 [SPARK-29930][SQL] Remove SQL configs declared to be removed in Spark 3.0
### What changes were proposed in this pull request?
In the PR, I propose to remove the following SQL configs:
1. `spark.sql.fromJsonForceNullableSchema`
2. `spark.sql.legacy.compareDateTimestampInTimestamp`
3. `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation`

that are declared to be removed in Spark 3.0

### Why are the changes needed?
To make code cleaner and improve maintainability.

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

### How was this patch tested?
By `TypeCoercionSuite`, `JsonExpressionsSuite` and `DDLSuite`.

Closes #26559 from MaxGekk/remove-sql-configs.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-17 10:14:04 -08:00
Dongjoon Hyun e1fc38b3e4 [SPARK-29932][R][TESTS] lint-r should do non-zero exit in case of errors
### What changes were proposed in this pull request?

This PR aims to make `lint-r` exits with non-zero in case of errors. Please note that `lint-r` works correctly when everything are installed correctly.

### Why are the changes needed?

There are two cases which hide errors from Jenkins/AppVeyor/GitHubAction.
1. `lint-r` exits with zero if there is no R installation.
```bash
$ dev/lint-r
dev/lint-r: line 25: type: Rscript: not found
ERROR: You should install R
$ echo $?
0
```

2. `lint-r` exits with zero if we didn't do `R/install-dev.sh`.
```bash
$ dev/lint-r
Error: You should install SparkR in a local directory with `R/install-dev.sh`.
In addition: Warning message:
In library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE) :
  no library trees found in 'lib.loc'
Execution halted
lintr checks passed.        // <=== Please note here
$ echo $?
0
```

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

No.

### How was this patch tested?

Manually check the above two cases.

Closes #26561 from dongjoon-hyun/SPARK-29932.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-17 10:09:46 -08:00
Pavithra Ramachandran a9959be2bc [SPARK-29456][WEBUI] Improve tooltip for Session Statistics Table column in JDBC/ODBC Server Tab
What changes were proposed in this pull request?
Some of the columns of JDBC/ODBC tab  Session info in Web UI are hard to understand.

Add tool tip for Start time, finish time , Duration and Total Execution

![Screenshot from 2019-10-16 12-33-17](https://user-images.githubusercontent.com/51401130/66901981-76d68980-f01d-11e9-9686-e20346a38c25.png)

Why are the changes needed?
To improve the understanding of the WebUI

Does this PR introduce any user-facing change?
No

How was this patch tested?
manual test

Closes #26138 from PavithraRamachandran/JDBC_tooltip.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-17 07:04:40 -06:00
fuwhu 388a737b98 [SPARK-29858][SQL] ALTER DATABASE (SET DBPROPERTIES) should look up catalog like v2 commands
### What changes were proposed in this pull request?
Add AlterNamespaceSetPropertiesStatement, AlterNamespaceSetProperties and AlterNamespaceSetPropertiesExec to make ALTER DATABASE (SET DBPROPERTIES) command look up catalog like v2 commands.

### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes, add "ALTER NAMESPACE ... SET (DBPROPERTIES | PROPERTIES) ..." whose function is same as "ALTER DATABASE ... SET DBPROPERTIES ..." and "ALTER SCHEMA ... SET DBPROPERTIES ...".

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

Closes #26551 from fuwhu/SPARK-29858.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-16 19:50:02 -08:00
Dongjoon Hyun cc12cf6029 [SPARK-29378][R] Upgrade SparkR to use Arrow 0.15 API
### What changes were proposed in this pull request?

[[SPARK-29376] Upgrade Apache Arrow to version 0.15.1](https://github.com/apache/spark/pull/26133) upgrades to Arrow 0.15 at Scala/Java/Python. This PR aims to upgrade `SparkR` to use Arrow 0.15 API. Currently, it's broken.

### Why are the changes needed?

First of all, it turns out that our Jenkins jobs (including PR builder) ignores Arrow test. Arrow 0.15 has a breaking R API changes at [ARROW-5505](https://issues.apache.org/jira/browse/ARROW-5505) and we missed that. AppVeyor was the only one having SparkR Arrow tests but it's broken now.

**Jenkins**
```
Skipped ------------------------------------------------------------------------
1. createDataFrame/collect Arrow optimization (test_sparkSQL_arrow.R#25)
- arrow not installed
```

Second, Arrow throws OOM on AppVeyor environment (Windows JDK8) like the following because it still has Arrow 0.14.
```
Warnings -----------------------------------------------------------------------
1. createDataFrame/collect Arrow optimization (test_sparkSQL_arrow.R#39) - createDataFrame attempted Arrow optimization because 'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, failed, attempting non-optimization. Reason: Error in handleErrors(returnStatus, conn): java.lang.OutOfMemoryError: Java heap space
	at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
	at java.nio.ByteBuffer.allocate(ByteBuffer.java:335)
	at org.apache.arrow.vector.ipc.message.MessageSerializer.readMessage(MessageSerializer.java:669)
	at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$3.readNextBatch(ArrowConverters.scala:243)
```

It is due to the version mismatch.
```java
int messageLength = MessageSerializer.bytesToInt(buffer.array());
if (messageLength == IPC_CONTINUATION_TOKEN) {
  buffer.clear();
  // ARROW-6313, if the first 4 bytes are continuation message, read the next 4 for the length
  if (in.readFully(buffer) == 4) {
    messageLength = MessageSerializer.bytesToInt(buffer.array());
  }
}

// Length of 0 indicates end of stream
if (messageLength != 0) {
  // Read the message into the buffer.
  ByteBuffer messageBuffer = ByteBuffer.allocate(messageLength);
```
 After upgrading this to 0.15, we are hitting ARROW-5505. This PR upgrades Arrow version in AppVeyor and fix the issue.

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

No.

### How was this patch tested?

Pass the AppVeyor.

This PR passed here.
- https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark/builds/28909044

```
SparkSQL Arrow optimization: Spark package found in SPARK_HOME: C:\projects\spark\bin\..
................
```

Closes #26555 from dongjoon-hyun/SPARK-R-TEST.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-16 18:28:27 -08:00
Maxim Gekk e88267cb5a [SPARK-29928][SQL][TESTS] Check parsing timestamps up to microsecond precision by JSON/CSV datasource
### What changes were proposed in this pull request?
In the PR, I propose to add tests from the commit 9c7e8be1dc for Spark 2.4 that check parsing of timestamp strings for various seconds fractions.

### Why are the changes needed?
To make sure that current behavior is the same as in Spark 2.4

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

### How was this patch tested?
By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`.

Closes #26558 from MaxGekk/parse-timestamp-micros-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-16 18:01:25 -08:00
Pavithra Ramachandran 5336473004 [SPARK-29476][WEBUI] add tooltip for Thread
### What changes were proposed in this pull request?
Adding tooltip for Thread Dump - Thread Locks

Before:
![Screenshot from 2019-11-04 17-11-22](https://user-images.githubusercontent.com/51401130/68127349-b963f580-ff3b-11e9-8547-e01907382632.png)

After:
![Screenshot from 2019-11-13 18-12-54](https://user-images.githubusercontent.com/51401130/68768698-08e7a700-0649-11ea-804b-2eb4d5f162b4.png)

### Why are the changes needed?
Thread Dump tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience.

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

### How was this patch tested?
Manual

Closes #26386 from PavithraRamachandran/threadDump_tooltip.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-16 13:20:05 -06:00
Dongjoon Hyun d0470d6394 [MINOR][TESTS] Ignore GitHub Action and AppVeyor file changes in testing
### What changes were proposed in this pull request?

This PR aims to ignore `GitHub Action` and `AppVeyor` file changes. When we touch these files, Jenkins job should not trigger a full testing.

### Why are the changes needed?

Currently, these files are categorized to `root` and trigger the full testing and ends up wasting the Jenkins resources.
- https://github.com/apache/spark/pull/26555
```
[info] Using build tool sbt with Hadoop profile hadoop2.7 under environment amplab_jenkins
From https://github.com/apache/spark
 * [new branch]      master     -> master
[info] Found the following changed modules: sparkr, root
[info] Setup the following environment variables for tests:
```

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

No. (Jenkins testing only).

### How was this patch tested?

Manually.
```
$ dev/run-tests.py -h -v
...
Trying:
    [x.name for x in determine_modules_for_files([".github/workflows/master.yml", "appveyor.xml"])]
Expecting:
    []
...
```

Closes #26556 from dongjoon-hyun/SPARK-IGNORE-APPVEYOR.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-16 09:26:01 -08:00
Yuanjian Li 40ea4a11d7 [SPARK-29807][SQL] Rename "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"
### What changes were proposed in this pull request?
Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"

### Why are the changes needed?
The relation between "spark.sql.ansi.enabled" and "spark.sql.dialect" is confusing, since the "PostgreSQL" dialect should contain the features of "spark.sql.ansi.enabled".

To make things clearer, we can rename the "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled", thus the option "spark.sql.dialect.spark.ansi.enabled" is only for Spark dialect.

For the casting and arithmetic operations, runtime exceptions should be thrown if "spark.sql.dialect" is "spark" and "spark.sql.dialect.spark.ansi.enabled" is true or "spark.sql.dialect" is PostgresSQL.

### Does this PR introduce any user-facing change?
Yes, the config name changed.

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

Closes #26444 from xuanyuanking/SPARK-29807.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-16 17:46:39 +08:00
Dongjoon Hyun f77c10de38 [SPARK-29923][SQL][TESTS] Set io.netty.tryReflectionSetAccessible for Arrow on JDK9+
### What changes were proposed in this pull request?

This PR aims to add `io.netty.tryReflectionSetAccessible=true` to the testing configuration for JDK11 because this is an officially documented requirement of Apache Arrow.

Apache Arrow community documented this requirement at `0.15.0` ([ARROW-6206](https://github.com/apache/arrow/pull/5078)).
> #### For java 9 or later, should set "-Dio.netty.tryReflectionSetAccessible=true".
> This fixes `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available`. thrown by netty.

### Why are the changes needed?

After ARROW-3191, Arrow Java library requires the property `io.netty.tryReflectionSetAccessible` to be set to true for JDK >= 9. After https://github.com/apache/spark/pull/26133, JDK11 Jenkins job seem to fail.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/676/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/677/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/678/

```scala
Previous exception in task:
sun.misc.Unsafe or java.nio.DirectByteBuffer.<init>(long, int) not available&#010;
io.netty.util.internal.PlatformDependent.directBuffer(PlatformDependent.java:473)&#010;
io.netty.buffer.NettyArrowBuf.getDirectBuffer(NettyArrowBuf.java:243)&#010;
io.netty.buffer.NettyArrowBuf.nioBuffer(NettyArrowBuf.java:233)&#010;
io.netty.buffer.ArrowBuf.nioBuffer(ArrowBuf.java:245)&#010;
org.apache.arrow.vector.ipc.message.ArrowRecordBatch.computeBodyLength(ArrowRecordBatch.java:222)&#010;
```

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

No.

### How was this patch tested?

Pass the Jenkins with JDK11.

Closes #26552 from dongjoon-hyun/SPARK-ARROW-JDK11.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 23:58:15 -08:00
Huaxin Gao 1112fc6029 [SPARK-29867][ML][PYTHON] Add __repr__ in Python ML Models
### What changes were proposed in this pull request?
Add ```__repr__``` in Python ML Models

### Why are the changes needed?
In Python ML Models, some of them have ```__repr__```, others don't. In the doctest, when calling Model.setXXX, some of the Models print out the xxxModel... correctly, some of them can't because of lacking the  ```__repr__``` method. For example:
```
    >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10)
    >>> model = gm.fit(df)
    >>> model.setPredictionCol("newPrediction")
    GaussianMixture...
```
After the change, the above code will become the following:
```
    >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10)
    >>> model = gm.fit(df)
    >>> model.setPredictionCol("newPrediction")
    GaussianMixtureModel...
```

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

### How was this patch tested?
doctest

Closes #26489 from huaxingao/spark-29876.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 21:44:39 -08:00
Takeshi Yamamuro 6d6b233791 [SPARK-29343][SQL][FOLLOW-UP] Remove floating-point Sum/Average/CentralMomentAgg from order-insensitive aggregates
### What changes were proposed in this pull request?

This pr is to remove floating-point `Sum/Average/CentralMomentAgg` from order-insensitive aggregates in `EliminateSorts`.

This pr comes from the gatorsmile suggestion: https://github.com/apache/spark/pull/26011#discussion_r344583899

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

Added tests in `SubquerySuite`.

Closes #26534 from maropu/SPARK-29343-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 18:54:02 -08:00
fuwhu 16e7195299 [SPARK-29834][SQL] DESC DATABASE should look up catalog like v2 commands
### What changes were proposed in this pull request?
Add DescribeNamespaceStatement, DescribeNamespace and DescribeNamespaceExec
to make "DESC DATABASE" look up catalog like v2 commands.

### Why are the changes needed?
It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes, add "DESC NAMESPACE" whose function is same as "DESC DATABASE" and "DESC SCHEMA".

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

Closes #26513 from fuwhu/SPARK-29834.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 18:50:42 -08:00
HyukjinKwon 7720781695 [SPARK-29127][SQL][PYTHON] Add a clue for Python related version information in integrated UDF tests
### What changes were proposed in this pull request?

This PR proposes to show Python, pandas and PyArrow versions in integrated UDF tests as a clue so when the test cases fail, it show the related version information.

I think we don't really need this kind of version information in the test case name for now since I intend that integrated SQL test cases do not target to test different combinations of Python, Pandas and PyArrow.

### Why are the changes needed?

To make debug easier.

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

It will change test name to include related Python, pandas and PyArrow versions.

### How was this patch tested?

Manually tested:

```
[info] - udf/postgreSQL/udf-case.sql - Scala UDF *** FAILED *** (8 seconds, 229 milliseconds)
[info]   udf/postgreSQL/udf-case.sql - Scala UDF
...
[info] - udf/postgreSQL/udf-case.sql - Regular Python UDF *** FAILED *** (6 seconds, 298 milliseconds)
[info]   udf/postgreSQL/udf-case.sql - Regular Python UDF
[info]   Python: 3.7
...
[info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF *** FAILED *** (6 seconds, 376 milliseconds)
[info]   udf/postgreSQL/udf-case.sql - Scalar Pandas UDF
[info]   Python: 3.7 Pandas: 0.25.3 PyArrow: 0.14.0
```

Closes #26538 from HyukjinKwon/investigate-flaky-test.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 18:37:33 -08:00
ulysses c0507e0f75 [SPARK-29833][YARN] Add FileNotFoundException check for spark.yarn.jars
### What changes were proposed in this pull request?

When set `spark.yarn.jars=/xxx/xxx` which is just a no schema path, spark will throw a NullPointerException.

The reason is hdfs will return null if pathFs.globStatus(path) is not exist, and spark just use `pathFs.globStatus(path).filter(_.isFile())` without check it.

### Why are the changes needed?

Avoid NullPointerException.

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

Yes. User will get a FileNotFoundException instead NullPointerException when `spark.yarn.jars` does not have schema and not exists.

### How was this patch tested?

Add UT.

Closes #26462 from ulysses-you/check-yarn-jars-path-exist.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-15 16:17:24 -08:00
Pablo Langa 848bdfa218 [SPARK-29829][SQL] SHOW TABLE EXTENDED should do multi-catalog resolution
### What changes were proposed in this pull request?

Add ShowTableStatement and make SHOW TABLE EXTENDED go through the same catalog/table resolution framework of v2 commands.

We don’t have this methods in the catalog to implement an V2 command

- catalog.getPartition
- catalog.getTempViewOrPermanentTableMetadata

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing

```sql
USE my_catalog
DESC t // success and describe the table t from my_catalog
SHOW TABLE EXTENDED FROM LIKE 't' // report table not found as there is no table t in the session catalog
```

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

Yes. When running SHOW TABLE EXTENDED Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.

### How was this patch tested?

Unit tests.

Closes #26540 from planga82/feature/SPARK-29481_ShowTableExtended.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-15 14:25:33 -08:00
shahid 15218898cd [SPARK-29902][DOC][MINOR] Add listener event queue capacity configuration to documentation
### What changes were proposed in this pull request?

Add listener event queue capacity configuration to documentation
### Why are the changes needed?

We some time see many event drops happening in eventLog listener queue. So, instead of increasing all the queues size, using this config we just need to increase eventLog queue capacity.

```
scala> sc.parallelize(1 to 100000, 100000).count()
[Stage 0:=================================================>(98299 + 4) / 100000]19/11/14 20:56:35 ERROR AsyncEventQueue: Dropping event from queue eventLog. This likely means one of the listeners is too slow and cannot keep up with the rate at which tasks are being started by the scheduler.
19/11/14 20:56:35 WARN AsyncEventQueue: Dropped 1 events from eventLog since the application started.
```

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

No

### How was this patch tested?

Existing tests

Closes #26529 from shahidki31/master1.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-15 08:20:10 -06:00
Takeshi Yamamuro ee4784bf26 [SPARK-26499][SQL][FOLLOW-UP] Replace update with setByte for ByteType in JdbcUtils.makeGetter
### What changes were proposed in this pull request?

This is a follow-up pr to fix the code coming from #23400; it replaces `update` with `setByte` for ByteType in `JdbcUtils.makeGetter`.

### Why are the changes needed?

For better code.

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

No.

### How was this patch tested?

Existing tests.

Closes #26532 from maropu/SPARK-26499-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-15 08:12:41 -06:00
Yuming Wang 4f10e54ba3 [SPARK-29655][SQL] Read bucketed tables obeys spark.sql.shuffle.partitions
### What changes were proposed in this pull request?

In order to avoid frequently changing the value of `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions`, we usually set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` much larger than `spark.sql.shuffle.partitions` after enabling adaptive execution, which causes some bucket map join lose efficacy and add more `ShuffleExchange`.

How to reproduce:
```scala
val bucketedTableName = "bucketed_table"
spark.range(10000).write.bucketBy(500, "id").sortBy("id").mode(org.apache.spark.sql.SaveMode.Overwrite).saveAsTable(bucketedTableName)
val bucketedTable = spark.table(bucketedTableName)
val df = spark.range(8)

spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1)
// Spark 2.4. spark.sql.adaptive.enabled=false
// We set spark.sql.shuffle.partitions <= 500 every time based on our data in this case.
spark.conf.set("spark.sql.shuffle.partitions", 500)
bucketedTable.join(df, "id").explain()
// Since 3.0. We enabled adaptive execution and set spark.sql.adaptive.shuffle.maxNumPostShufflePartitions to a larger values to fit more cases.
spark.conf.set("spark.sql.adaptive.enabled", true)
spark.conf.set("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions", 1000)
bucketedTable.join(df, "id").explain()
```

```
scala> bucketedTable.join(df, "id").explain()
== Physical Plan ==
*(4) Project [id#5L]
+- *(4) SortMergeJoin [id#5L], [id#7L], Inner
   :- *(1) Sort [id#5L ASC NULLS FIRST], false, 0
   :  +- *(1) Project [id#5L]
   :     +- *(1) Filter isnotnull(id#5L)
   :        +- *(1) ColumnarToRow
   :           +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 500 out of 500
   +- *(3) Sort [id#7L ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(id#7L, 500), true, [id=#49]
         +- *(2) Range (0, 8, step=1, splits=16)
```
vs
```
scala> bucketedTable.join(df, "id").explain()
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- Project [id#5L]
   +- SortMergeJoin [id#5L], [id#7L], Inner
      :- Sort [id#5L ASC NULLS FIRST], false, 0
      :  +- Exchange hashpartitioning(id#5L, 1000), true, [id=#93]
      :     +- Project [id#5L]
      :        +- Filter isnotnull(id#5L)
      :           +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 500 out of 500
      +- Sort [id#7L ASC NULLS FIRST], false, 0
         +- Exchange hashpartitioning(id#7L, 1000), true, [id=#92]
            +- Range (0, 8, step=1, splits=16)
```

This PR makes read bucketed tables always obeys `spark.sql.shuffle.partitions` even enabling adaptive execution and set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` to avoid add more `ShuffleExchange`.

### Why are the changes needed?
Do not degrade performance after enabling adaptive execution.

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

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

Closes #26409 from wangyum/SPARK-29655.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-15 15:49:24 +08:00