Commit graph

25153 commits

Author SHA1 Message Date
Yuming Wang adb506afd7 [SPARK-28852][SQL] Implement SparkGetCatalogsOperation for Thrift Server
### What changes were proposed in this pull request?
This PR implements `SparkGetCatalogsOperation` for Thrift Server metadata completeness.

### Why are the changes needed?
Thrift Server metadata completeness.

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

### How was this patch tested?
Unit test

Closes #25555 from wangyum/SPARK-28852.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-08-25 22:42:50 -07:00
Terry Kim a3328cdc0a [SPARK-28238][SQL][FOLLOW-UP] Clean up attributes for Datasource v2 DESCRIBE TABLE
### What changes were proposed in this pull request?
1. Fix the physical plan (`DescribeTableExec`) to have the same output attributes as the corresponding logical plan.
2. Remove `output` in statements since they are unresolved plans.

### Why are the changes needed?
Correctness of how output attributes should work.

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

### How was this patch tested?
Existing tests

Closes #25568 from imback82/describe_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-26 13:39:36 +08:00
Dongjoon Hyun 6214b6a541 [SPARK-28868][INFRA] Specify Jekyll version to 3.8.6 in release docker image
### What changes were proposed in this pull request?

This PR aims to specify Jekyll Version explicitly in our release docker image.

### Why are the changes needed?
Recently, Jekyll 4.0 is released and it dropped Ruby 2.3 support.
This breaks our release docker image build.
```
Building native extensions.  This could take a while...
ERROR:  Error installing jekyll:
        jekyll-sass-converter requires Ruby version >= 2.4.0.
```

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

No.

### How was this patch tested?

The following should succeed.
```
$ docker build -t spark-rm:test --build-arg UID=501 dev/create-release/spark-rm
...
Successfully tagged spark-rm:test
```

Closes #25578 from dongjoon-hyun/SPARK-28868.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-25 15:38:41 -07:00
Yuming Wang 4b16cf11b3 [SPARK-27988][SQL][TEST] Port AGGREGATES.sql [Part 3]
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/aggregates.out#L986-L1613

When porting the test cases, found seven PostgreSQL specific features that do not exist in Spark SQL:

[SPARK-27974](https://issues.apache.org/jira/browse/SPARK-27974): Add built-in Aggregate Function: array_agg
[SPARK-27978](https://issues.apache.org/jira/browse/SPARK-27978): Add built-in Aggregate Functions: string_agg
[SPARK-27986](https://issues.apache.org/jira/browse/SPARK-27986): Support Aggregate Expressions with filter
[SPARK-27987](https://issues.apache.org/jira/browse/SPARK-27987): Support POSIX Regular Expressions
[SPARK-28682](https://issues.apache.org/jira/browse/SPARK-28682): ANSI SQL: Collation Support
[SPARK-28768](https://issues.apache.org/jira/browse/SPARK-28768): Implement more text pattern operators
[SPARK-28865](https://issues.apache.org/jira/browse/SPARK-28865): Table inheritance

## How was this patch tested?

N/A

Closes #24829 from wangyum/SPARK-27988.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-25 23:34:59 +09:00
Yuming Wang 02a0cdea13 [SPARK-28723][SQL] Upgrade to Hive 2.3.6 for HiveMetastore Client and Hadoop-3.2 profile
### What changes were proposed in this pull request?

This PR upgrade the built-in Hive to 2.3.6 for `hadoop-3.2`.

Hive 2.3.6 release notes:
- [HIVE-22096](https://issues.apache.org/jira/browse/HIVE-22096): Backport [HIVE-21584](https://issues.apache.org/jira/browse/HIVE-21584) (Java 11 preparation: system class loader is not URLClassLoader)
- [HIVE-21859](https://issues.apache.org/jira/browse/HIVE-21859): Backport [HIVE-17466](https://issues.apache.org/jira/browse/HIVE-17466) (Metastore API to list unique partition-key-value combinations)
- [HIVE-21786](https://issues.apache.org/jira/browse/HIVE-21786): Update repo URLs in poms branch 2.3 version

### Why are the changes needed?
Make Spark support JDK 11.

### Does this PR introduce any user-facing change?
Yes. Please see [SPARK-28684](https://issues.apache.org/jira/browse/SPARK-28684) and [SPARK-24417](https://issues.apache.org/jira/browse/SPARK-24417) for more details.

### How was this patch tested?
Existing unit test and manual test.

Closes #25443 from wangyum/test-on-jenkins.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-23 21:34:30 -07:00
Anton Kirillov f17f1d01e2 [SPARK-28778][MESOS] Fixed executors advertised address when running in virtual network
### What changes were proposed in this pull request?
Resolves [SPARK-28778: Shuffle jobs fail due to incorrect advertised address when running in a virtual network on Mesos](https://issues.apache.org/jira/browse/SPARK-28778).

This patch fixes a bug which occurs when shuffle jobs are launched by Mesos in a virtual network. Mesos scheduler sets executor `--hostname` parameter to `0.0.0.0` in the case when `spark.mesos.network.name` is provided. This makes executors use `0.0.0.0` as their advertised address and, in the presence of shuffle, executors fail to fetch shuffle blocks from each other using `0.0.0.0` as the origin. When a virtual network is used the hostname or IP address is not known upfront and assigned to a container at its start time so the executor process needs to advertise the correct dynamically assigned address to be reachable by other executors.

Changes:
- added a fallback to `Utils.localHostName()` in Spark Executors when `--hostname` is not provided
- removed setting executor address to `0.0.0.0` from Mesos scheduler
- refactored the code related to building executor command in Mesos scheduler
- added network configuration support to Docker containerizer
- added unit tests

### Why are the changes needed?
The bug described above prevents Mesos users from running any jobs which involve shuffle due to the inability of executors to fetch shuffle blocks because of incorrect advertised address when virtual network is used.

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

### How was this patch tested?
- added unit test to `MesosCoarseGrainedSchedulerBackendSuite` which verifies the absence of `--hostname` parameter  when `spark.mesos.network.name` is provided and its presence otherwise
- added unit test to `MesosSchedulerBackendUtilSuite` which verifies that `MesosSchedulerBackendUtil.buildContainerInfo` sets network-related properties for Docker containerizer
- unit tests from this repo launched with profiles: `./build/mvn test -Pmesos -Pnetlib-lgpl -Psparkr -Phive -Phive-thriftserver`, build log attached: [mvn.test.log](https://github.com/apache/spark/files/3516891/mvn.test.log)
- integration tests from [DCOS Spark repo](https://github.com/mesosphere/spark-build), more specifically - [test_spark_cni.py](https://github.com/mesosphere/spark-build/blob/master/tests/test_spark_cni.py) which runs a specific [shuffle job](https://github.com/mesosphere/spark-build/blob/master/tests/jobs/scala/src/main/scala/ShuffleApp.scala) and verifies its successful completion, Mesos task network configuration, and IP addresses for both Mesos and Docker containerizers

Closes #25500 from akirillov/DCOS-45840-fix-advertised-ip-in-virtual-networks.

Authored-by: Anton Kirillov <akirillov@mesosophere.io>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-23 18:30:05 -07:00
zhengruifeng 573b1cb835 [SPARK-28858][ML][PYSPARK] add tree-based transformation in the py side
### What changes were proposed in this pull request?
expose the newly added tree-based transformation in the py side

### Why are the changes needed?
function parity

### Does this PR introduce any user-facing change?
yes, add `setLeafCol` & `getLeafCol` in the py side

### How was this patch tested?
added tests & local tests

Closes #25566 from zhengruifeng/py_tree_path.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-08-23 15:18:35 -07:00
HyukjinKwon d25cbd44ee [SPARK-28839][CORE] Avoids NPE in context cleaner when dynamic allocation and shuffle service are on
### What changes were proposed in this pull request?

This PR proposes to avoid to thrown NPE at context cleaner when shuffle service is on - it is kind of a small followup of https://github.com/apache/spark/pull/24817

Seems like it sets `null` for `shuffleIds` to track when the service is on. Later, `removeShuffle` tries to remove an element at `shuffleIds` which leads to NPE. It fixes it by explicitly not sending the event (`ShuffleCleanedEvent`) in this case.

See the code path below:

cbad616d4c/core/src/main/scala/org/apache/spark/SparkContext.scala (L584)

cbad616d4c/core/src/main/scala/org/apache/spark/ContextCleaner.scala (L125)

cbad616d4c/core/src/main/scala/org/apache/spark/ContextCleaner.scala (L190)

cbad616d4c/core/src/main/scala/org/apache/spark/ContextCleaner.scala (L220-L230)

cbad616d4c/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala (L353-L357)

cbad616d4c/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala (L347)

cbad616d4c/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala (L400-L406)

cbad616d4c/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala (L475)

cbad616d4c/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala (L427)

### Why are the changes needed?

This is a bug fix.

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

It prevents the exception:

```
19/08/21 06:44:01 ERROR AsyncEventQueue: Listener ExecutorMonitor threw an exception
java.lang.NullPointerException
	at org.apache.spark.scheduler.dynalloc.ExecutorMonitor$Tracker.removeShuffle(ExecutorMonitor.scala:479)
	at org.apache.spark.scheduler.dynalloc.ExecutorMonitor.$anonfun$cleanupShuffle$2(ExecutorMonitor.scala:408)
	at org.apache.spark.scheduler.dynalloc.ExecutorMonitor.$anonfun$cleanupShuffle$2$adapted(ExecutorMonitor.scala:407)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
	at org.apache.spark.scheduler.dynalloc.ExecutorMonitor.cleanupShuffle(ExecutorMonitor.scala:407)
	at org.apache.spark.scheduler.dynalloc.ExecutorMonitor.onOtherEvent(ExecutorMonitor.sc
```

### How was this patch test?

Unittest was added.

Closes #25551 from HyukjinKwon/SPARK-28839.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-08-23 12:44:56 -07:00
Xiao Li 07c4b9bd1f Revert "[SPARK-25474][SQL] Support spark.sql.statistics.fallBackToHdfs in data source tables"
This reverts commit 485ae6d181.

Closes #25563 from gatorsmile/revert.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-23 07:41:39 -07:00
Gengliang Wang 8258660f67 [SPARK-28741][SQL] Optional mode: throw exceptions when casting to integers causes overflow
## What changes were proposed in this pull request?

To follow ANSI SQL, we should support a configurable mode that throws exceptions when casting to integers causes overflow.
The behavior is similar to https://issues.apache.org/jira/browse/SPARK-26218, which throws exceptions on arithmetical operation overflow.
To unify it, the configuration is renamed from "spark.sql.arithmeticOperations.failOnOverFlow" to "spark.sql.failOnIntegerOverFlow"
## How was this patch tested?

Unit test

Closes #25461 from gengliangwang/AnsiCastIntegral.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-23 21:49:45 +08:00
Ali Afroozeh 1472e664ba [SPARK-28716][SQL] Add id to Exchange and Subquery's stringArgs method for easier identifying their reuses in query plans
## What changes were proposed in this pull request?

Add id to Exchange and Subquery's stringArgs method for easier identifying their reuses in query plans, for example:
```
ReusedExchange d_date_sk#827, BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))) [id=#2710]
```
Where `2710` is the id of the reused exchange.

## How was this patch tested?

Passes existing tests

Closes #25434 from dbaliafroozeh/ImplementStringArgsExchangeSubqueryExec.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-08-23 13:29:32 +02:00
Ali Afroozeh aef7ca1f0b [SPARK-28836][SQL] Remove the canonicalize(attributes) method from PlanExpression
### What changes were proposed in this pull request?
This PR removes the `canonicalize(attrs: AttributeSeq)` from `PlanExpression` and taking care of normalizing expressions in `QueryPlan`.

### Why are the changes needed?
`Expression` has already a `canonicalized` method and having the `canonicalize` method in `PlanExpression` is confusing.

### Does this PR introduce any user-facing change?
Removes the `canonicalize` plan from `PlanExpression`. Also renames the `normalizeExprId` to `normalizeExpressions` in query plan.

### How was this patch tested?
This PR is a refactoring and passes the existing tests

Closes #25534 from dbaliafroozeh/ImproveCanonicalizeAPI.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-08-23 13:26:58 +02:00
Dongjoon Hyun 1fd7f290ab [SPARK-28857][INFRA] Clean up the comments of PR template during merging
### What changes were proposed in this pull request?

This PR aims to clean up the commit logs by removing the comments of our PR template.

### Why are the changes needed?

Apache Spark PR template has comments. Sometime we forget to clean up them because GitHub hides them nicely. It would be great if we clean up this. Otherwise, this makes the commit logs too verbose. (There are a few commits already.)

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

No. (only for committers)

### How was this patch tested?

Manually with Python2/Python3.

Closes #25564 from dongjoon-hyun/SPARK-28857.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-23 18:08:10 +09:00
terryk 98e1a4cea4 [SPARK-28319][SQL] Implement SHOW TABLES for Data Source V2 Tables
## What changes were proposed in this pull request?

Implements the SHOW TABLES logical and physical plans for data source v2 tables.

## How was this patch tested?

Added unit tests to `DataSourceV2SQLSuite`.

Closes #25247 from imback82/dsv2_show_tables.

Lead-authored-by: terryk <yuminkim@gmail.com>
Co-authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-23 14:20:25 +08:00
Ali Afroozeh 9976b876f1 [SPARK-28835][SQL][TEST] Add TPCDSSchema trait
### What changes were proposed in this pull request?
This PR extracts the schema information of TPCDS tables into a separate class called `TPCDSSchema` which can be reused for other testing purposes

### How was this patch tested?
This PR is only a refactoring for tests and passes existing tests

Closes #25535 from dbaliafroozeh/IntroduceTPCDSSchema.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-22 23:18:46 -07:00
Jungtaek Lim (HeartSaVioR) 406c5331ff
[SPARK-28025][SS] Fix FileContextBasedCheckpointFileManager leaking crc files
### What changes were proposed in this pull request?

This PR fixes the leak of crc files from CheckpointFileManager when FileContextBasedCheckpointFileManager is being used.

Spark hits the Hadoop bug, [HADOOP-16255](https://issues.apache.org/jira/browse/HADOOP-16255) which seems to be a long-standing issue.

This is there're two `renameInternal` methods:

```
public void renameInternal(Path src, Path dst)
public void renameInternal(final Path src, final Path dst, boolean overwrite)
```

which should be overridden to handle all cases but ChecksumFs only overrides method with 2 params, so when latter is called FilterFs.renameInternal(...) is called instead, and it will do rename with RawLocalFs as underlying filesystem.

The bug is related to FileContext, so FileSystemBasedCheckpointFileManager is not affected.

[SPARK-17475](https://issues.apache.org/jira/browse/SPARK-17475) took a workaround for this bug, but [SPARK-23966](https://issues.apache.org/jira/browse/SPARK-23966) seemed to bring regression.

This PR deletes crc file as "best-effort" when renaming, as failing to delete crc file is not that critical to fail the task.

### Why are the changes needed?

This PR prevents crc files not being cleaned up even purging batches. Too many files in same directory often hurts performance, as well as each crc file occupies more space than its own size so possible to occupy nontrivial amount of space when batches go up to 100000+.

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

No.

### How was this patch tested?

Some unit tests are modified to check leakage of crc files.

Closes #25488 from HeartSaVioR/SPARK-28025.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-08-22 23:10:16 -07:00
Gengliang Wang 895c90b582 [SPARK-28730][SQL] Configurable type coercion policy for table insertion
## What changes were proposed in this pull request?

After all the discussions in the dev list: http://apache-spark-developers-list.1001551.n3.nabble.com/Discuss-Follow-ANSI-SQL-on-table-insertion-td27531.html#a27562.
Here I propose that we can make the store assignment rules in the analyzer configurable, and the behavior of V1 and V2 should be consistent.
When inserting a value into a column with a different data type, Spark will perform type coercion. After this PR, we support 2 policies for the type coercion rules:
legacy and strict.
1. With legacy policy, Spark allows casting any value to any data type. The legacy policy is the only behavior in Spark 2.x and it is compatible with Hive.
2. With strict policy, Spark doesn't allow any possible precision loss or data truncation in type coercion, e.g. `int` and `long`, `float` -> `double` are not allowed.

Eventually, the "legacy" mode will be removed, so it is disallowed in data source V2.
To ensure backward compatibility with existing queries, the default store assignment policy for data source V1 is "legacy".
## How was this patch tested?

Unit test

Closes #25453 from gengliangwang/tableInsertRule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-23 13:50:26 +08:00
shivusondur 23bed0d3c0 [SPARK-28702][SQL] Display useful error message (instead of NPE) for invalid Dataset operations
### What changes were proposed in this pull request?
Added proper message instead of NPE for invalid Dataset operations (e.g. calling actions inside of transformations) similar to SPARK-5063 for RDD

### Why are the changes needed?
To report the user about the exact issue instead of NPE

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

### How was this patch tested?

Manually tested

```scala
test code snap
"import spark.implicits._
    val ds1 = spark.sparkContext.parallelize(1 to 100, 100).toDS()
    val ds2 = spark.sparkContext.parallelize(1 to 100, 100).toDS()
    ds1.map(x => {
      // scalastyle:off
      println(ds2.count + x)
      x
    }).collect()"
```

Closes #25503 from shivusondur/jira28702.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-08-22 22:15:37 -07:00
Kousuke Saruta 33e45ec7b8 [SPARK-28769][CORE] Improve warning message of BarrierExecutionMode when required slots > maximum slots
### What changes were proposed in this pull request?
Improved warning message in Barrier Execution Mode when required slots > maximum slots.
The new message contains information about required slots, maximum slots and how many times retry failed.

### Why are the changes needed?
Providing to users with the number of required slots, maximum slots and how many times retry failed might help users to decide what they should do.
For example, continuing to wait for retry succeeded or killing jobs.

### Does this PR introduce any user-facing change?
Yes.
If `spark.scheduler.barrier.maxConcurrentTaskCheck.maxFailures=3`, we get following warning message.

Before applying this change:

```
19/08/18 15:18:09 WARN DAGScheduler: The job 2 requires to run a barrier stage that requires more slots than the total number of slots in the cluster currently.
19/08/18 15:18:24 WARN DAGScheduler: The job 2 requires to run a barrier stage that requires more slots than the total number of slots in the cluster currently.
19/08/18 15:18:39 WARN DAGScheduler: The job 2 requires to run a barrier stage that requires more slots than the total number of slots in the cluster currently.
19/08/18 15:18:54 WARN DAGScheduler: The job 2 requires to run a barrier stage that requires more slots than the total number of slots in the cluster currently.
org.apache.spark.scheduler.BarrierJobSlotsNumberCheckFailed: [SPARK-24819]: Barrier execution mode does not allow run a barrier stage that requires more slots than the total number of slots in the cluster currently. Please init a new cluster with more CPU cores or repartition the input RDD(s) to reduce the number of slots required to run this barrier stage.
  at org.apache.spark.scheduler.DAGScheduler.checkBarrierStageWithNumSlots(DAGScheduler.scala:439)
  at org.apache.spark.scheduler.DAGScheduler.createResultStage(DAGScheduler.scala:453)
  at org.apache.spark.scheduler.DAGScheduler.handleJobSubmitted(DAGScheduler.scala:983)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2140)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2132)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2121)
  at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
  at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:749)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2080)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2101)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2120)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2145)
  at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:961)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.rdd.RDD.withScope(RDD.scala:366)
  at org.apache.spark.rdd.RDD.collect(RDD.scala:960)
  ... 47 elided
```
After applying this change:

```
19/08/18 16:52:23 WARN DAGScheduler: The job 0 requires to run a barrier stage that requires 3 slots than the total number of slots(2) in the cluster currently.
19/08/18 16:52:38 WARN DAGScheduler: The job 0 requires to run a barrier stage that requires 3 slots than the total number of slots(2) in the cluster currently (Retry 1/3 failed).
19/08/18 16:52:53 WARN DAGScheduler: The job 0 requires to run a barrier stage that requires 3 slots than the total number of slots(2) in the cluster currently (Retry 2/3 failed).
19/08/18 16:53:08 WARN DAGScheduler: The job 0 requires to run a barrier stage that requires 3 slots than the total number of slots(2) in the cluster currently (Retry 3/3 failed).
org.apache.spark.scheduler.BarrierJobSlotsNumberCheckFailed: [SPARK-24819]: Barrier execution mode does not allow run a barrier stage that requires more slots than the total number of slots in the cluster currently. Please init a new cluster with more CPU cores or repartition the input RDD(s) to reduce the number of slots required to run this barrier stage.
  at org.apache.spark.scheduler.DAGScheduler.checkBarrierStageWithNumSlots(DAGScheduler.scala:439)
  at org.apache.spark.scheduler.DAGScheduler.createResultStage(DAGScheduler.scala:453)
  at org.apache.spark.scheduler.DAGScheduler.handleJobSubmitted(DAGScheduler.scala:983)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2140)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2132)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2121)
  at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
  at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:749)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2080)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2101)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2120)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2145)
  at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:961)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.rdd.RDD.withScope(RDD.scala:366)
  at org.apache.spark.rdd.RDD.collect(RDD.scala:960)
  ... 47 elided
```

### How was this patch tested?
I tested manually using Spark Shell with following configuration and script. And then, checked log message.

```
$ bin/spark-shell --master local[2] --conf spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures=3
scala> sc.parallelize(1 to 100, sc.defaultParallelism+1).barrier.mapPartitions(identity(_)).collect
```

Closes #25487 from sarutak/barrier-exec-mode-warning-message.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-22 14:06:58 -05:00
zhengruifeng defb65ed1a [SPARK-13677][ML] Implement Tree-Based Feature Transformation for ML
## What changes were proposed in this pull request?
Tree-based feature transformation is a widely used feature and already implemented in many famous libraries, like sklearn/xgboost/lightgbm/catboost. But is still missing in ML.
The previous discussions and design doc can be found in [SPARK-13677](https://issues.apache.org/jira/browse/SPARK-13677), which is the only left subtask in 'GBT improvement umbrella' [SPARK-14047](https://issues.apache.org/jira/browse/SPARK-14047).

This pr is to add tree-based feature transformation.

## How was this patch tested?
existing and added suites

Closes #25383 from zhengruifeng/tree_path.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-22 09:37:42 -05:00
heleny fb1f868d4f [SPARK-28776][ML] SparkML Writer gets hadoop conf from session state
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
SparkML writer gets hadoop conf from session state, instead of the spark context.
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->

### Why are the changes needed?
Allow for multiple sessions in the same context that have different hadoop configurations.
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
No

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
Tested in pyspark.ml.tests.test_persistence.PersistenceTest test_default_read_write

Closes #25505 from helenyugithub/SPARK-28776.

Authored-by: heleny <heleny@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-22 09:27:05 -05:00
Dongjoon Hyun 36da2e3384 [SPARK-28847][TEST] Annotate HiveExternalCatalogVersionsSuite with ExtendedHiveTest
### What changes were proposed in this pull request?

This PR aims to annotate `HiveExternalCatalogVersionsSuite` with `ExtendedHiveTest`.

### Why are the changes needed?

`HiveExternalCatalogVersionsSuite` is an outstanding test in terms of testing time. This PR aims to allow skipping this test suite when we use `ExtendedHiveTest`.
![time](https://user-images.githubusercontent.com/9700541/63489184-4c75af00-c466-11e9-9e12-d250d4a23292.png)

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

No

### How was this patch tested?

Since Jenkins doesn't exclude `ExtendedHiveTest`, there is no difference in Jenkins testing.
This PR should be tested by manually by the following.

**BEFORE**
```
$ cd sql/hive
$ mvn package -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest
...
Run starting. Expected test count is: 1
HiveExternalCatalogVersionsSuite:
22:32:16.218 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load ...
```

**AFTER**
```
$ cd sql/hive
$ mvn package -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest
...
Run starting. Expected test count is: 0
HiveExternalCatalogVersionsSuite:
Run completed in 772 milliseconds.
Total number of tests run: 0
Suites: completed 2, aborted 0
Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0
No tests were executed.
...
```

Closes #25550 from dongjoon-hyun/SPARK-28847.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-22 00:25:56 -07:00
triplesheep 48578a41b5 [SPARK-28844][SQL] Fix typo in SQLConf FILE_COMRESSION_FACTOR
### What changes were proposed in this pull request?
Fix minor typo in SQLConf.
`FILE_COMRESSION_FACTOR` -> `FILE_COMPRESSION_FACTOR`

### Why are the changes needed?
Make conf more understandable.

### Does this PR introduce any user-facing change?
No. (`spark.sql.sources.fileCompressionFactor` is unchanged.)

### How was this patch tested?
Pass the Jenkins with the existing tests.

Closes #25538 from triplesheep/TYPO-FIX.

Authored-by: triplesheep <triplesheep0419@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-22 00:07:40 -07:00
Sean Owen 9ea37b09cf [SPARK-17875][CORE][BUILD] Remove dependency on Netty 3
### What changes were proposed in this pull request?

Spark uses Netty 4 directly, but also includes Netty 3 only because transitive dependencies do. The dependencies (Hadoop HDFS, Zookeeper, Avro) don't seem to need this dependency as used in Spark. I think we can forcibly remove it to slim down the dependencies.

Previous attempts were blocked by its usage in Flume, but that dependency has gone away.
https://github.com/apache/spark/pull/15436

### Why are the changes needed?

Mostly to reduce the transitive dependency size and complexity a little bit and avoid triggering spurious security alerts on Netty 3.x usage.

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

No

### How was this patch tested?

Existing tests

Closes #25544 from srowen/SPARK-17875.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-21 21:27:56 -07:00
maryannxue aefb2e70e7 [SPARK-28739][SQL] Add a simple cost check for Adaptive Query Execution
### What changes were proposed in this pull request?

This PR adds a simple cost model and a mechanism to compare the costs of the before and after plans of each re-optimization in Adaptive Query Execution. Now the workflow of AQE re-optimization is changed to: If the cost of the plan after re-optimization is lower than or equal to that of the plan before re-optimization and the plan has been changed after re-optimization (if equal), the current physical plan will be updated to the plan after re-optimization, otherwise it will remain unchanged until the next re-optimization.

### Why are the changes needed?
This new mechanism is to prevent regressions in Adaptive Query Execution caused by change of the plan introducing extra cost, in this PR specifically, change of SMJ to BHJ leading to extra `ShuffleExchangeExec`s.

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

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

Closes #25456 from maryannxue/aqe-cost.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-21 19:33:56 -07:00
Wenchen Fan ed3ea6734c [SPARK-28837][SQL] CTAS/RTAS should use nullable schema
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
When running CTAS/RTAS, use the nullable schema of the input query to create the table.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
It's very likely to run CTAS/RTAS with non-nullable input query, e.g. `CREATE TABLE t AS SELECT 1`. However, it's surprising to users if they can't write null to this table later. Non-nullable is kind of a constraint of the column and should be specified by users explicitly.

For reference, Postgres also use nullable schema for CTAS:
```
> create table t1(i int not null);

> insert into t1 values (1);

> create table t2 as select i from t1;

> \d+ t1;
 Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
 i      | integer |           | not null |         | plain   |              |

> \d+ t2;
 Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
 i      | integer |           |          |         | plain   |              |

```

File source V1 has the same behavior.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
Yes, after this PR CTAS/RTAS creates tables with nullable schema, then users can insert null values later.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
new test

Closes #25536 from cloud-fan/ctas.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-22 09:49:18 +08:00
Wenchen Fan 97b046f06f [SPARK-28635][SQL][FOLLOWUP] CatalogManager should reflect the changes of default catalog
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
The current namespace/catalog should be set to None at the beginning, so that we can read the new configs when reporting currennt namespace/catalog later.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
Fix a bug in CatalogManager, to reflect the change of default catalog config when reporting current catalog.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
No. The current namespace/catalog stuff is still internal right now.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
a new test suite

Closes #25521 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-21 12:23:42 -07:00
Yuanjian Li 2d9cc42aa8 [SPARK-28699][SQL] Disable using radix sort for ShuffleExchangeExec in repartition case
## What changes were proposed in this pull request?

Disable using radix sort in ShuffleExchangeExec when we do repartition.
In #20393, we fixed the indeterministic result in the shuffle repartition case by performing a local sort before repartitioning.
But for the newly added sort operation, we use radix sort which is wrong because binary data can't be compared by only the prefix. This makes the sort unstable and fails to solve the indeterminate shuffle output problem.

### Why are the changes needed?
Fix the correctness bug caused by repartition after a shuffle.

### Does this PR introduce any user-facing change?
Yes, user will get the right result in the case of repartition stage rerun.

## How was this patch tested?

Test with `local-cluster[5, 2, 5120]`, use the integrated test below, it can return a right answer 100000000.
```
import scala.sys.process._
import org.apache.spark.TaskContext

val res = spark.range(0, 10000 * 10000, 1).map{ x => (x % 1000, x)}
// kill an executor in the stage that performs repartition(239)
val df = res.repartition(113).map{ x => (x._1 + 1, x._2)}.repartition(239).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 1 && TaskContext.get.stageAttemptNumber == 0) {
    throw new Exception("pkill -f -n java".!!)
  }
  x
}
val r2 = df.distinct.count()
```

Closes #25491 from xuanyuanking/SPARK-28699-fix.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-21 10:56:50 -07:00
Ali Afroozeh 4dc3093513 [SPARK-28715][SQL] Introduce collectInPlanAndSubqueries and subqueriesAll in QueryPlan
## What changes were proposed in this pull request?

Introduces the collectInPlanAndSubqueries and subqueriesAll methods in QueryPlan that consider all the plans in the query plan, including the ones in nested subqueries.

## How was this patch tested?

Unit test added

Closes #25433 from dbaliafroozeh/IntroduceCollectInPlanAndSubqueries.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-08-21 18:05:18 +02:00
zhengruifeng bdef7125b7 [SPARK-28540][WEBUI] Document Environment page
## What changes were proposed in this pull request?
Document Environment page

## How was this patch tested?
locally building

![图片](https://user-images.githubusercontent.com/7322292/63237759-e3c7e000-c275-11e9-8e1f-57ed1b0e86e8.png)

Closes #25430 from zhengruifeng/doc_ui_conf.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-21 10:48:48 -05:00
zhengruifeng 49ffbff2fc [SPARK-28780][ML] Delete the incorrect setWeightCol method in LinearSVCModel
### What changes were proposed in this pull request?
Delete the incorrect method `def setWeightCol(value: Double): this.type = set(threshold, value)` in `LinearSVCModel`

### Why are the changes needed?
`LinearSVCModel` should not provide this setter, moreover, this method is wrongly defined.

### Does this PR introduce any user-facing change?
yes, a public method is removed

### How was this patch tested?
existing suites

Closes #25510 from zhengruifeng/linearsvc_model_set_weightcol.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-21 09:47:53 -05:00
WeichenXu 9779a82ea0 [SPARK-28483][CORE][FOLLOW-UP] Dealing with interrupted exception in BarrierTaskContext.barrier()
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
Dealing with interrupted exception in BarrierTaskContext.barrier()

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
Interrupted exception will happen in the case sparkContext local property "spark.job.interruptOnCancel" set true.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
No.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
UT.

Closes #25519 from WeichenXu123/barrier_fl.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-21 19:51:45 +08:00
Robert (Bobby) Evans fac469e2e0 [SPARK-28774][SQL] Fix exchange reuse for columnar data
### What changes were proposed in this pull request?
The rule ReuseExchange optimization rule will look for instances of Exchange that have the same plan and convert dedupe them to them to a ReuseExchangeExec instance. In the current Spark codebase all Exchange instances are row based, but if we use the spark.sql.extensions config to put in our own columnar based exchange implementation reuse will throw an exception saying that there was a columnar mismatch.

### Why are the changes needed?
Without it Reused Columnar Exchanges throw an exception

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

### How was this patch tested?

I tested this patch by running it against a query that was showing this exact issue and it fixed it.

I also added a very simple unit test that shows the issue.

Closes #25499 from revans2/reused-columnar-exchange.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-21 18:10:26 +08:00
Burak Yavuz 4855bfe16b [SPARK-28554][SQL] Adds a v1 fallback writer implementation for v2 data source codepaths
## What changes were proposed in this pull request?

This PR adds a V1 fallback interface for writing to V2 Tables using V1 Writer interfaces. The only supported SaveMode that will be called on the target table will be an Append. The target table must use V2 interfaces such as `SupportsOverwrite` or `SupportsTruncate` to support Overwrite operations. It is up to the target DataSource implementation if this operation can be atomic or not.

We do not support dynamicPartitionOverwrite, as we cannot call a `commit` method that actually cleans up the data in the partitions that were touched through this fallback.

## How was this patch tested?

Will add tests and example implementation after comments + feedback. This is a proposal at this point.

Closes #25348 from brkyvz/v1WriteFallback.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-21 17:25:25 +08:00
zhengruifeng c4257b18a1 [SPARK-28541][WEBUI] Document Storage page
## What changes were proposed in this pull request?
add an example for storage tab

## How was this patch tested?
locally building

Closes #25445 from zhengruifeng/doc_ui_storage.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-20 20:05:13 -05:00
Marco Gaido 0bfcf9c210 [SPARK-28322][SQL] Add support to Decimal type for integral divide
## What changes were proposed in this pull request?

The expression `IntegralDivide`, which corresponds to the `div` operator, support only integral type. Postgres, though, allows it to work also with decimals.

The PR adds the support to decimal operands for this operation in order to have feature parity with postgres.

## How was this patch tested?

added UTs

Closes #25136 from mgaido91/SPARK-28322.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-21 08:43:00 +09:00
Dhruve Ashar a50959a7f6 [SPARK-27937][CORE] Revert partial logic for auto namespace discovery
## What changes were proposed in this pull request?
This change reverts the logic which was introduced as a part of SPARK-24149 and a subsequent followup PR.

With existing logic:
- Spark fails to launch with HDFS federation enabled while trying to get a path to a logical nameservice.
- It gets tokens for unrelated namespaces if they are used in HDFS Federation
- Automatic namespace discovery is supported only if these are on the same cluster.

Rationale for change:
- For accessing data from related namespaces, viewfs should handle getting tokens for spark
- For accessing data from unrelated namespaces(user explicitly specifies them using existing configs) as these could be on the same or different cluster.

(Please fill in changes proposed in this fix)
Revert the changes.

## How was this patch tested?
Ran few manual tests and unit test.

Closes #24785 from dhruve/bug/SPARK-27937.

Authored-by: Dhruve Ashar <dhruveashar@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-08-20 12:42:35 -07:00
maryannxue 39c11273e0 [SPARK-28753][SQL] Dynamically reuse subqueries in AQE
### What changes were proposed in this pull request?
This PR changes subquery reuse in Adaptive Query Execution from compile-time static reuse to execution-time dynamic reuse. This PR adds a `ReuseAdaptiveSubquery` rule that applies to a query stage after it is created and before it is executed. The new dynamic reuse enables subqueries to be reused across all different subquery levels.

### Why are the changes needed?
This is an improvement to the current subquery reuse in Adaptive Query Execution, which allows subquery reuse to happen in a lazy fashion as well as at different subquery levels.

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

### How was this patch tested?
Passed existing tests.

Closes #25471 from maryannxue/aqe-dynamic-sub-reuse.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 19:58:29 +08:00
Wenchen Fan d04522187a [SPARK-28635][SQL] create CatalogManager to track registered v2 catalogs
## What changes were proposed in this pull request?

This is a pure refactor PR, which creates a new class `CatalogManager` to track the registered v2 catalogs, and provide the catalog up functionality.

`CatalogManager` also tracks the current catalog/namespace. We will implement corresponding commands in other PRs, like `USE CATALOG my_catalog`

## How was this patch tested?

existing tests

Closes #25368 from cloud-fan/refactor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 19:40:21 +08:00
Jungtaek Lim (HeartSaVioR) b37c8d5cea
[SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter
#  What changes were proposed in this pull request?

This patch modifies the explanation of guarantee for ForeachWriter as it doesn't guarantee same output for `(partitionId, epochId)`. Refer the description of [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for more details.

Spark itself still guarantees same output for same epochId (batch) if the preconditions are met, 1) source is always providing the same input records for same offset request. 2) the query is idempotent in overall (indeterministic calculation like now(), random() can break this).

Assuming breaking preconditions as an exceptional case (the preconditions are implicitly required even before), we still can describe the guarantee with `epochId`, though it will be  harder to leverage the guarantee: 1) ForeachWriter should implement a feature to track whether all the partitions are written successfully for given `epochId` 2) There's pretty less chance to leverage the fact, as the chance for Spark to successfully write all partitions and fail to checkpoint the batch is small.

Credit to zsxwing on discovering the broken guarantee.

## How was this patch tested?

This is just a documentation change, both on javadoc and guide doc.

Closes #25407 from HeartSaVioR/SPARK-28650.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-08-20 00:56:53 -07:00
lihao 79464bed2f [SPARK-28662][SQL] Create Hive Partitioned Table DDL should fail when partition column type missed
## What changes were proposed in this pull request?
Create Hive Partitioned Table without specifying data type for partition column will success unexpectedly.
```HiveQL
// create a hive table partition by b, but the data type of b isn't specified.
CREATE TABLE tbl(a int) PARTITIONED BY (b) STORED AS parquet
```
In https://issues.apache.org/jira/browse/SPARK-26435 ,  PARTITIONED BY clause  are extended to support Hive CTAS as following:
```ANTLR
// Before
(PARTITIONED BY '(' partitionColumns=colTypeList ')'

 // After
(PARTITIONED BY '(' partitionColumns=colTypeList ')'|
PARTITIONED BY partitionColumnNames=identifierList) |
```

Create Table Statement like above case will pass the syntax check,  and recognized as (PARTITIONED BY partitionColumnNames=identifierList) 。

This PR  will check this case in visitCreateHiveTable and throw a exception which contains  explicit error message to user.

## How was this patch tested?

Added tests.

Closes #25390 from lidinghao/hive-ddl-fix.

Authored-by: lihao <lihaowhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 14:37:04 +08:00
WeichenXu bc75ed675b [SPARK-28483][CORE] Fix canceling a spark job using barrier mode but barrier tasks blocking on BarrierTaskContext.barrier()
## What changes were proposed in this pull request?

Fix canceling a spark job using barrier mode but barrier tasks do not exit.
Currently, when spark tasks are killed, `BarrierTaskContext.barrier()` cannot be killed (it will blocking on RPC request), cause the task blocking and cannot exit.

In my PR I implement an interface for RPC which support `abort` in class `RpcEndpointRef`
```
  def askAbortable[T: ClassTag](
      message: Any,
      timeout: RpcTimeout): AbortableRpcFuture[T]
```

The returned `AbortableRpcFuture` instance include an `abort` method so that we can abort the RPC before it timeout.

## How was this patch tested?

Unit test added.

Manually test:

### Test code
launch spark-shell via `spark-shell --master local[4]`
and run following code:
```
sc.setLogLevel("INFO")
import org.apache.spark.BarrierTaskContext
val n = 4
def taskf(iter: Iterator[Int]): Iterator[Int] = {
  val context = BarrierTaskContext.get()
  val x = iter.next()
  if (x % 2 == 0) {
    // sleep 6000000 seconds with task killed checking
    for (i <- 0 until 6000000) {
      Thread.sleep(1000)
      if (context.isInterrupted()) {
        throw new org.apache.spark.TaskKilledException()
      }
    }
  }
  context.barrier()
  return Iterator.empty
}

// launch spark job, including 4 tasks, tasks 1/3 will enter `barrier()`, and tasks 0/2 will enter `sleep`
sc.parallelize((0 to n), n).barrier().mapPartitions(taskf).collect()
```
And then press Ctrl+C to exit the running job.

### Before
press Ctrl+C to exit the running job, then open spark UI we can see 2 tasks (task 1/3) are not killed. They are blocking.

### After
press Ctrl+C to exit the running job,  we can see in spark UI all tasks killed successfully.

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #25235 from WeichenXu123/sc_14848.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 14:21:47 +08:00
Yuanjian Li 0d3a783cc5 [SPARK-28699][CORE] Fix a corner case for aborting indeterminate stage
### What changes were proposed in this pull request?
Change the logic of collecting the indeterminate stage, we should look at stages from mapStage, not failedStage during handle FetchFailed.

### Why are the changes needed?
In the fetch failed error handle logic, the original logic of collecting indeterminate stage from the fetch failed stage. And in the scenario of the fetch failed happened in the first task of this stage, this logic will cause the indeterminate stage to resubmit partially. Eventually, we are capable of getting correctness bug.

### Does this PR introduce any user-facing change?
It makes the corner case of indeterminate stage abort as expected.

### How was this patch tested?
New UT in DAGSchedulerSuite.
Run below integrated test with `local-cluster[5, 2, 5120]`, and set `spark.sql.execution.sortBeforeRepartition`=false, it will abort the indeterminate stage as expected:
```
import scala.sys.process._
import org.apache.spark.TaskContext

val res = spark.range(0, 10000 * 10000, 1).map{ x => (x % 1000, x)}
// kill an executor in the stage that performs repartition(239)
val df = res.repartition(113).map{ x => (x._1 + 1, x._2)}.repartition(239).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 1 && TaskContext.get.stageAttemptNumber == 0) {
    throw new Exception("pkill -f -n java".!!)
  }
  x
}
val r2 = df.distinct.count()
```

Closes #25498 from xuanyuanking/SPARK-28699-followup.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-20 13:47:59 +08:00
darrentirto a787bc2884 [SPARK-28777][PYTHON][DOCS] Fix format_string doc string with the correct parameters
### What changes were proposed in this pull request?
The parameters doc string of the function format_string was changed from _col_, _d_ to _format_, _cols_ which is what the actual function declaration states

### Why are the changes needed?
The parameters stated by the documentation was inaccurate

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

**BEFORE**
![before](https://user-images.githubusercontent.com/9700541/63310013-e21a0e80-c2ad-11e9-806b-1d272c5cde12.png)

**AFTER**
![after](https://user-images.githubusercontent.com/9700541/63315812-6b870c00-c2c1-11e9-8165-82782628cd1a.png)

### How was this patch tested?
N/A: documentation only
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->

Closes #25506 from darrentirto/SPARK-28777.

Authored-by: darrentirto <darrentirto@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-19 20:44:46 -07:00
Sean Owen 3b4e345fa1 [SPARK-28775][CORE][TESTS] Skip date 8633 in Kwajalein due to changes in tzdata2018i that only some JDK 8s use
### What changes were proposed in this pull request?

Some newer JDKs use the tzdata2018i database, which changes how certain (obscure) historical dates and timezones are handled. As previously, we can pretty much safely ignore these in tests, as the value may vary by JDK.

### Why are the changes needed?

Test otherwise fails using, for example, JDK 1.8.0_222. https://bugs.openjdk.java.net/browse/JDK-8215982 has a full list of JDKs which has this.

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

No.

### How was this patch tested?

Existing tests

Closes #25504 from srowen/SPARK-28775.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-19 17:54:25 -07:00
Mick Jermsurawong b79cf0d143 [SPARK-28224][SQL] Check overflow in decimal Sum aggregate
## What changes were proposed in this pull request?
- Currently `sum` in aggregates for decimal type can overflow and return null.
  - `Sum` expression codegens arithmetic on `sql.Decimal` and the output which preserves scale and precision goes into `UnsafeRowWriter`. Here overflowing will be converted to null when writing out.
  - It also does not go through this branch in `DecimalAggregates` because it's expecting precision of the sum (not the elements to be summed) to be less than 5.
4ebff5b6d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala (L1400-L1403)

- This PR adds the check at the final result of the sum operator itself.
4ebff5b6d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala (L372-L376)

https://issues.apache.org/jira/browse/SPARK-28224

## How was this patch tested?

- Added an integration test on dataframe suite

cc mgaido91 JoshRosen

Closes #25033 from mickjermsurawong-stripe/SPARK-28224.

Authored-by: Mick Jermsurawong <mickjermsurawong@stripe.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-20 09:47:04 +09:00
Takuya UESHIN 26f344354b [SPARK-27905][SQL][FOLLOW-UP] Add prettyNames
### What changes were proposed in this pull request?

This is a follow-up of #24761 which added a higher-order function `ArrayForAll`.
The PR mistakenly removed the `prettyName` from `ArrayExists` and forgot to add it to `ArrayForAll`.

### Why are the changes needed?

This reverts the `prettyName` back to `ArrayExists` not to affect explained plans, and adds it to `ArrayForAll` to clarify the `prettyName` as the same as the expressions around.

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

No.

### How was this patch tested?

Existing tests.

Closes #25501 from ueshin/issues/SPARK-27905/pretty_names.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-19 15:15:50 -07:00
Sean Owen fa7fd8f2a4 [SPARK-28434][TESTS][ML] Fix values in dummy tree in DecisionTreeSuite
### What changes were proposed in this pull request?

Fix dummy tree created in decision tree tests to have actually consistent stats, so that it can be compared in tests more completely. The current one has values for, say, impurity that don't even match internally.

With this, the tests can assert more about stats staying correct after load.

### Why are the changes needed?

Fixes a TODO and improves the test slightly.

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

None

### How was this patch tested?

Existing tests.

Closes #25485 from srowen/SPARK-28434.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-19 17:01:14 -05:00
Marcelo Vanzin 5f6eb5d20d [SPARK-28634][YARN] Ignore kerberos login config in client mode AM
This change makes the client mode AM ignore any login configuration,
which is now always handled by the driver. The previous code tried
to achieve that by modifying the configuration visible to the AM, but
that missed the case where old configuration names were being used.

Tested in real cluster with reproduction provided in the bug.

Closes #25467 from vanzin/SPARK-28634.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-08-19 11:06:02 -07:00
HyukjinKwon 1de4a22c52 Revert "[SPARK-28759][BUILD] Upgrade scala-maven-plugin to 4.1.1"
This reverts commit 1819a6f22e.
2019-08-19 20:32:07 +09:00