Commit graph

7901 commits

Author SHA1 Message Date
Brandon Jiang 1450b5e095 [MINOR][DOCS] fix typo for docs,log message and comments
### What changes were proposed in this pull request?
Fix typo for docs, log messages and comments

### Why are the changes needed?
typo fix to increase readability

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

### How was this patch tested?
manual test has been performed to test the updated

Closes #29443 from brandonJY/spell-fix-doc.

Authored-by: Brandon Jiang <Brandon.jiang.a@outlook.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-22 06:45:35 +09:00
yi.wu 44a288fc41 [SPARK-32653][CORE] Decommissioned host/executor should be considered as inactive in TaskSchedulerImpl
### What changes were proposed in this pull request?

Add decommissioning status checking for a host or executor while checking it's active or not. And a decommissioned host or executor should be considered as inactive.

### Why are the changes needed?

First of all, this PR is not a correctness bug fix but gives improvement indeed. And the main problem here we want to fix is that a decommissioned host or executor should be considered as inactive.

`TaskSetManager.computeValidLocalityLevels` depends on `TaskSchedulerImpl.isExecutorAlive/hasExecutorsAliveOnHost` to calculate the locality levels. Therefore, the `TaskSetManager` could also get corresponding locality levels of those decommissioned hosts or executors if they're not considered as inactive. However, on the other side,  `CoarseGrainedSchedulerBackend` won't construct the `WorkerOffer` for those decommissioned executors. That also means `TaskSetManager` might never have a chance to launch tasks at certain locality levels but only suffers the unnecessary delay because of delay scheduling. So, this PR helps to reduce this kind of unnecessary delay by making decommissioned host/executor inactive in `TaskSchedulerImpl`.

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

No.

### How was this patch tested?

Added unit tests

Closes #29468 from Ngone51/fix-decom-alive.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-20 12:00:32 +00:00
Xingbo Jiang f793977e9a [SPARK-32658][CORE] Fix PartitionWriterStream partition length overflow
### What changes were proposed in this pull request?

The `count` in `PartitionWriterStream` should be a long value, instead of int. The issue is introduced by apache/sparkabef84a . When the overflow happens, the shuffle index file would record wrong index of a reduceId, thus lead to `FetchFailedException: Stream is corrupted` error.

Besides the fix, I also added some debug logs, so in the future it's easier to debug similar issues.

### Why are the changes needed?

This is a regression and bug fix.

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

No

### How was this patch tested?

A Spark user reported this issue when migrating their workload to 3.0. One of the jobs fail deterministically on Spark 3.0 without the patch, and the job succeed after applied the fix.

Closes #29474 from jiangxb1987/fixPartitionWriteStream.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-20 07:08:30 +00:00
yi.wu a1a32d2eb5 [SPARK-32600][CORE] Unify task name in some logs between driver and executor
### What changes were proposed in this pull request?

This PR replaces some arbitrary task names in logs with the widely used task name (e.g. "task 0.0 in stage 1.0 (TID 1)") among driver and executor. This will change the task name in `TaskDescription` by appending TID.

### Why are the changes needed?

Some logs are still using TID(a.k.a `taskId`) only as the task name, e.g.,

7f275ee597/core/src/main/scala/org/apache/spark/executor/Executor.scala (L786)

7f275ee597/core/src/main/scala/org/apache/spark/executor/Executor.scala (L632-L635)

And the task thread name also only has the `taskId`:

7f275ee597/core/src/main/scala/org/apache/spark/executor/Executor.scala (L325)

As mentioned in https://github.com/apache/spark/pull/1259, TID itself does not capture stage or retries, making it harder to correlate with the application. It's inconvenient when debugging applications.

Actually, task name like "task name (e.g. "task 0.0 in stage 1.0 (TID 1)")" has already been used widely after https://github.com/apache/spark/pull/1259. We'd better follow the naming convention.

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

Yes. Users will see the more consistent task names in the log.

### How was this patch tested?

Manually checked.

Closes #29418 from Ngone51/unify-task-name.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-19 08:44:49 +00:00
yi.wu 3092527f75 [SPARK-32651][CORE] Decommission switch configuration should have the highest hierarchy
### What changes were proposed in this pull request?

Rename `spark.worker.decommission.enabled` to `spark.decommission.enabled` and move it from `org.apache.spark.internal.config.Worker` to `org.apache.spark.internal.config.package`.

### Why are the changes needed?

Decommission has been supported in Standalone and k8s yet and may be supported in Yarn(https://github.com/apache/spark/pull/27636) in the future. Therefore, the switch configuration should have the highest hierarchy rather than belongs to Standalone's Worker. In other words, it should be independent of the cluster managers.

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

No, as the decommission feature hasn't been released.

### How was this patch tested?

Pass existed tests.

Closes #29466 from Ngone51/fix-decom-conf.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-19 06:53:06 +00:00
yi.wu 70964e741a [SPARK-21040][CORE][FOLLOW-UP] Only calculate executorKillTime when speculation is enabled
### What changes were proposed in this pull request?

Only calculate `executorKillTime` in `TaskSetManager.executorDecommission()` when speculation is enabled.

### Why are the changes needed?

Avoid unnecessary operations to save time/memory.

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

No.

### How was this patch tested?

Pass existed tests.

Closes #29464 from Ngone51/followup-SPARK-21040.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-18 13:50:57 +00:00
Devesh Agrawal 1ac23dea52 [SPARK-32613][CORE] Fix regressions in DecommissionWorkerSuite
### What changes were proposed in this pull request?

The DecommissionWorkerSuite started becoming flaky and it revealed a real regression. Recently closed #29211 necessitates remembering the decommissioning shortly beyond the removal of the executor.

In addition to fixing this issue, ensure that DecommissionWorkerSuite continues to pass when executors haven't had a chance to exit eagery. That is the old behavior before #29211 also still works.

Added some more tests to TaskSchedulerImpl to ensure that the decommissioning information is indeed purged after a timeout.

Hardened the test DecommissionWorkerSuite to make it wait for successful job completion.

### Why are the changes needed?

First, let me describe the intended behavior of decommissioning: If a fetch failure happens where the source executor was decommissioned, we want to treat that as an eager signal to clear all shuffle state associated with that executor. In addition if we know that the host was decommissioned, we want to forget about all map statuses from all other executors on that decommissioned host. This is what the test "decommission workers ensure that fetch failures lead to rerun" is trying to test. This invariant is important to ensure that decommissioning a host does not lead to multiple fetch failures that might fail the job. This fetch failure can happen before the executor is truly marked "lost" because of heartbeat delays.

- However, #29211 eagerly exits the executors when they are done decommissioning. This removal of the executor was racing with the fetch failure. By the time the fetch failure is triggered the executor is already removed and thus has forgotten its decommissioning information. (I tested this by delaying the decommissioning). The fix is to keep the decommissioning information around for some time after removal with some extra logic to finally purge it after a timeout.

- In addition the executor loss can also bump up `shuffleFileLostEpoch` (added in #28848). This happens because when the executor is lost, it forgets the shuffle state about just that executor and increments the `shuffleFileLostEpoch`. This incrementing precludes the clearing of state of the entire host when the fetch failure happens because the failed task is still reusing the old epoch. The fix here is also simple: Ignore the `shuffleFileLostEpoch` when the shuffle status is being cleared due to a fetch failure resulting from host decommission.

I am strategically making both of these fixes be very local to decommissioning to avoid other regressions. Especially the version stuff is tricky (it hasn't been fundamentally changed since it was first introduced in 2013).

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

### How was this patch tested?
Manually ran DecommissionWorkerSuite several times using a script and ensured it all passed.

### (Internal) Configs added
I added two configs, one of which is sort of meant for testing only:
- `spark.test.executor.decommission.initial.sleep.millis`: Initial delay by the decommissioner shutdown thread. Default is same as before of 1 second. This is used for testing only. This one is kept "hidden" (ie not added as a constant to avoid config bloat)
- `spark.executor.decommission.removed.infoCacheTTL`: Number of seconds to keep the removed executors decom entries around. It defaults to 5 minutes. It should be around the average time it takes for all of the shuffle data to be fetched from the mapper to the reducer, but I think that can take a while since the reducers also do a multistep sort.

Closes #29422 from agrawaldevesh/decom_fixes.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-18 06:47:31 +00:00
Cheng Su 8f0fef1843 [SPARK-32399][SQL] Full outer shuffled hash join
### What changes were proposed in this pull request?

Add support for full outer join inside shuffled hash join. Currently if the query is a full outer join, we only use sort merge join as the physical operator. However it can be CPU and IO intensive in case input table is large for sort merge join. Shuffled hash join on the other hand saves the sort CPU and IO compared to sort merge join, especially when table is large.

This PR implements the full outer join as followed:
* Process rows from stream side by looking up hash relation, and mark the matched rows from build side by:
  * for joining with unique key, a `BitSet` is used to record matched rows from build side (`key index` to represent each row)
  * for joining with non-unique key, a `HashSet[Long]` is  used to record matched rows from build side (`key index` + `value index` to represent each row).
`key index` is defined as the index into key addressing array `longArray` in `BytesToBytesMap`.
`value index` is defined as the iterator index of values for same key.

* Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`)

For context, this PR was originally implemented as followed (up to commit e3322766d4):
1. Construct hash relation from build side, with extra boolean value at the end of row to track look up information (done in `ShuffledHashJoinExec.buildHashedRelation` and `UnsafeHashedRelation.apply`).
2. Process rows from stream side by looking up hash relation, and mark the matched rows from build side be looked up (done in `ShuffledHashJoinExec.fullOuterJoin`).
3. Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`).

See discussion of pros and cons between these two approaches [here](https://github.com/apache/spark/pull/29342#issuecomment-672275450), [here](https://github.com/apache/spark/pull/29342#issuecomment-672288194) and [here](https://github.com/apache/spark/pull/29342#issuecomment-672640531).

TODO: codegen for full outer shuffled hash join can be implemented in another followup PR.

### Why are the changes needed?

As implementation in this PR, full outer shuffled hash join will have overhead to iterate build side twice (once for building hash map, and another for outputting non-matching rows), and iterate stream side once. However, full outer sort merge join needs to iterate both sides twice, and sort the large table can be more CPU and IO intensive. So full outer shuffled hash join can be more efficient than sort merge join when stream side is much more larger than build side.

For example query below, full outer SHJ saved 30% wall clock time compared to full outer SMJ.

```
def shuffleHashJoin(): Unit = {
    val N: Long = 4 << 22
    withSQLConf(
      SQLConf.SHUFFLE_PARTITIONS.key -> "2",
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000") {
      codegenBenchmark("shuffle hash join", N) {
        val df1 = spark.range(N).selectExpr(s"cast(id as string) as k1")
        val df2 = spark.range(N / 10).selectExpr(s"cast(id * 10 as string) as k2")
        val df = df1.join(df2, col("k1") === col("k2"), "full_outer")
        df.noop()
    }
  }
}
```

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join off
  Stopped after 2 iterations, 16602 ms
  Running case: shuffle hash join on
  Stopped after 5 iterations, 31911 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join off                              7900           8301         567          2.1         470.9       1.0X
shuffle hash join on                               6250           6382          95          2.7         372.5       1.3X
```

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

No.

### How was this patch tested?

Added unit test in `JoinSuite.scala`, `AbstractBytesToBytesMapSuite.java` and `HashedRelationSuite.scala`.

Closes #29342 from c21/full-outer-shj.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-17 08:06:19 +09:00
Kousuke Saruta 1a4c8f718f [SPARK-32119][CORE] ExecutorPlugin doesn't work with Standalone Cluster and Kubernetes with --jars
### What changes were proposed in this pull request?

This PR changes Executor to load jars and files added by --jars and --files on Executor initialization.
To avoid downloading those jars/files twice, they are assosiated with `startTime` as their uploaded timestamp.

### Why are the changes needed?

ExecutorPlugin can't work with Standalone Cluster and Kubernetes
when a jar which contains plugins and files used by the plugins are added by --jars and --files option with spark-submit.

This is because jars and files added by --jars and --files are not loaded on Executor initialization.
I confirmed it works with YARN because jars/files are distributed as distributed cache.

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

Yes. jars/files added by --jars and --files are downloaded on each executor on initialization.

### How was this patch tested?

Added a new testcase.

Closes #28939 from sarutak/fix-plugin-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-08-14 17:10:22 -05:00
Holden Karau 548ac7c4af [SPARK-31198][CORE] Use graceful decommissioning as part of dynamic scaling
### What changes were proposed in this pull request?

If graceful decommissioning is enabled, Spark's dynamic scaling uses this instead of directly killing executors.

### Why are the changes needed?

When scaling down Spark we should avoid triggering recomputes as much as possible.

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

Hopefully their jobs run faster or at the same speed. It also enables experimental shuffle service free dynamic scaling when graceful decommissioning is enabled (using the same code as the shuffle tracking dynamic scaling).

### How was this patch tested?

For now I've extended the ExecutorAllocationManagerSuite for both core & streaming.

Closes #29367 from holdenk/SPARK-31198-use-graceful-decommissioning-as-part-of-dynamic-scaling.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-08-12 17:07:18 -07:00
yi.wu c6ea98323f [SPARK-32250][SPARK-27510][CORE][TEST] Fix flaky MasterSuite.test(...) in Github Actions
### What changes were proposed in this pull request?

Set more dispatcher threads for the flaky test.

### Why are the changes needed?

When running test on Github Actions machine, the available processors in JVM  is only 2, while on Jenkins it's 32. For this specific test, 2 available processors, which also decides the number of threads in Dispatcher, are not enough to consume the messages. In the worst situation, `MockExecutorLaunchFailWorker` would occupy these 2 threads for handling messages `LaunchDriver`, `LaunchExecutor` at the same time but leave no thread for the driver to handle the message `RegisteredApplication`. At the end, it results in a deadlock situation and causes the test failure.

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

No.

### How was this patch tested?

We can check whether the test is still flaky in Github Actions after this fix.

Closes #29408 from Ngone51/spark-32250.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 21:05:50 +09:00
Venkata krishnan Sowrirajan 2d6eb00256 [SPARK-32596][CORE] Clear Ivy resolution files as part of finally block
<!--
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.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->

### What changes were proposed in this pull request?
Clear Ivy resolution files as part of finally block if not failures while artifacts resolution can leave the resolution files around.
Use tempIvyPath for SparkSubmitUtils.buildIvySettings in tests. This why the test
"SPARK-10878: test resolution files cleaned after resolving artifact" did not capture these issues.

### Why are the changes needed?
This is a bug

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

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

Closes #29411 from venkata91/SPARK-32596.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-08-12 00:22:22 -05:00
Maryann Xue 9587277717 [SPARK-32470][CORE] Remove task result size check for shuffle map stage
### What changes were proposed in this pull request?

This PR removes the total task result size check for shuffle map stage tasks, as these tasks return map status and metrics, which will not be cached on the driver and thus will not crash the driver.

### Why are the changes needed?

Checking total task result size for shuffle map stage tasks would lead to erroring normal jobs which create a big number of tasks even if the job eventually does not return a large dataset.

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

No.

### How was this patch tested?

Added UT.

Closes #29276 from maryannxue/spark-32470.

Authored-by: Maryann Xue <maryann.xue@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 06:29:32 +00:00
Dongjoon Hyun b421bf0196 [SPARK-32517][CORE] Add StorageLevel.DISK_ONLY_3
### What changes were proposed in this pull request?

This PR aims to add `StorageLevel.DISK_ONLY_3` as a built-in `StorageLevel`.

### Why are the changes needed?

In a YARN cluster, HDFS uaually provides storages with replication factor 3. So, we can save the result to HDFS to get `StorageLevel.DISK_ONLY_3` technically. However, disaggregate clusters or clusters without storage services are rising. Previously, in that situation, the users were able to use similar `MEMORY_AND_DISK_2` or a user-created `StorageLevel`. This PR aims to support those use cases officially for better UX.

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

Yes. This provides a new built-in option.

### How was this patch tested?

Pass the GitHub Action or Jenkins with the revised test cases.

Closes #29331 from dongjoon-hyun/SPARK-32517.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-10 07:33:06 -07:00
Devesh Agrawal 34c61b9548 [SPARK-32575][CORE][TESTS] Bump up timeouts in BlockManagerDecommissionIntegrationSuite to reduce flakyness
### What changes were proposed in this pull request?

As reported by HyukjinKwon, BlockManagerDecommissionIntegrationSuite test is apparently still flaky (even after https://github.com/apache/spark/pull/29226): https://github.com/apache/spark/pull/29226#issuecomment-670286829.

The new flakyness is because the executors are not launching in the 6 seconds time out I had given them when run under github checks.

Bumped up the timeouts.

### Why are the changes needed?

To make this test not flaky so that it can give us high signal if decommissioning regresses.

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

No, unit test only check.

### How was this patch tested?

No new tests. Just github and jenkins.

Closes #29388 from agrawaldevesh/more_bm_harden.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-09 20:00:39 -07:00
Yan Xiaole cf37cd518e [SPARK-32557][CORE] Logging and swallowing the exception per entry in History server
### What changes were proposed in this pull request?
This PR adds a try catch wrapping the History server scan logic to log and swallow the exception per entry.

### Why are the changes needed?
As discussed in #29350 , one entry failure shouldn't affect others.

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

### How was this patch tested?
Manually tested.

Closes #29374 from yanxiaole/SPARK-32557.

Authored-by: Yan Xiaole <xiaole.yan@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-09 16:47:31 -07:00
Kousuke Saruta 34d9f1cf4c [SPARK-32462][WEBUI] Reset previous search text for datatable
### What changes were proposed in this pull request?

This PR proposes to change the behavior of DataTable for stage-page and executors-page not to save the previous search text.

### Why are the changes needed?

DataTable is used in stage-page and executors-page for pagination and filter tasks/executors by search text.
In the current implementation, search text is saved so if we visit stage-page for a job, the previous search text is filled in the textbox and the task table is filtered.
I'm sometimes surprised by this behavior as the stage-page lists no tasks because tasks are filtered by the previous search text.
I think, it's not useful.

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

Yes. Search text is no longer saved.

### How was this patch tested?

New testcase with the following command.
```
$ build/sbt -Dguava.version=27.0-jre -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite -- -z Search"
```

Closes #29265 from sarutak/fix-search-box.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 22:14:29 -07:00
JoeyValentine dc3fac8184 [MINOR][DOCS] Fix typos at ExecutorAllocationManager.scala
### What changes were proposed in this pull request?

This PR fixes some typos in <code>core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala</code> file.

### Why are the changes needed?

<code>spark.dynamicAllocation.sustainedSchedulerBacklogTimeout</code> (N) is used only after the <code>spark.dynamicAllocation.schedulerBacklogTimeout</code> (M) is exceeded.

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

No.

### How was this patch tested?

No test needed.

Closes #29351 from JoeyValentine/master.

Authored-by: JoeyValentine <rlaalsdn0506@naver.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 12:36:07 -07:00
Kousuke Saruta 4e267f3eb9 [SPARK-32538][CORE][TEST] Use local time zone for the timestamp logged in unit-tests.log
### What changes were proposed in this pull request?

This PR lets the logger log timestamp based on local time zone during test.
`SparkFunSuite` fixes the default time zone to America/Los_Angeles so the timestamp logged in unit-tests.log is also based on the fixed time zone.

### Why are the changes needed?

It's confusable for developers whose time zone is not America/Los_Angeles.

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

No.

### How was this patch tested?

Run existing tests and confirmed uint-tests.log.
If your local time zone is America/Los_Angeles, you can test by setting the environment variable `TZ` like as follows.
```
$ TZ=Asia/Tokyo build/sbt "testOnly org.apache.spark.executor.ExecutorSuite"
$ tail core/target/unit-tests.log
```

Closes #29356 from sarutak/fix-unit-test-log-timezone.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-07 11:29:18 +09:00
Gengliang Wang e93b8f02cd [SPARK-32539][INFRA] Disallow FileSystem.get(Configuration conf) in style check by default
### What changes were proposed in this pull request?

Disallow `FileSystem.get(Configuration conf)` in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem()` instead.

### Why are the changes needed?

The method `FileSystem.get(Configuration conf)` will return a default FileSystem instance if the conf `fs.file.impl` is not set. This can cause file not found exception on reading a target path of non-default file system, e.g. S3. It is hard to discover such a mistake via unit tests.
If we disallow it in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem(Configuration conf)`, we can reduce potential regression and PR review effort.

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

No

### How was this patch tested?

Manually run scala style check and test.

Closes #29357 from gengliangwang/newStyleRule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 05:56:59 +00:00
yi.wu 7f275ee597 [SPARK-32518][CORE] CoarseGrainedSchedulerBackend.maxNumConcurrentTasks should consider all kinds of resources
### What changes were proposed in this pull request?

1.  Make `CoarseGrainedSchedulerBackend.maxNumConcurrentTasks()` considers all kinds of resources when calculating the max concurrent tasks

2. Refactor `calculateAvailableSlots()` to make it be able to be used for both `CoarseGrainedSchedulerBackend` and `TaskSchedulerImpl`

### Why are the changes needed?

Currently, `CoarseGrainedSchedulerBackend.maxNumConcurrentTasks()` only considers the CPU for the max concurrent tasks. This can cause the application to hang when a barrier stage requires extra custom resources but the cluster doesn't have enough corresponding resources. Because, without the checking for other custom resources in `maxNumConcurrentTasks`, the barrier stage can be submitted to the `TaskSchedulerImpl`. But the `TaskSchedulerImpl` won't launch tasks for the barrier stage due to the insufficient task slots calculated by `TaskSchedulerImpl.calculateAvailableSlots` (which does check all kinds of resources).

The application hang issue can be reproduced by the added unit test.

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

Yes. In case of a barrier stage requires more custom resources than the cluster has, the application can get hang before this PR but can fail due to insufficient resources at the end after this PR.

### How was this patch tested?

Added a unit test.

Closes #29332 from Ngone51/fix-slots.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 05:39:47 +00:00
Holden Karau 375d348a83 [SPARK-31197][CORE] Shutdown executor once we are done decommissioning
### What changes were proposed in this pull request?

Exit the executor when it has been asked to decommission and there is nothing left for it to do.

This is a rebase of https://github.com/apache/spark/pull/28817

### Why are the changes needed?

If we want to use decommissioning in Spark's own scale down we should terminate the executor once finished.
Furthermore, in graceful shutdown it makes sense to release resources we no longer need if we've been asked to shutdown by the cluster manager instead of always holding the resources as long as possible.

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

The decommissioned executors will exit and the end of decommissioning. This is sort of a user facing change, however decommissioning hasn't been in any releases yet.

### How was this patch tested?

I changed the unit test to not send the executor exit message and still wait on the executor exited message.

Closes #29211 from holdenk/SPARK-31197-exit-execs-redone.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-08-05 16:28:14 -07:00
Yan Xiaole c1d17df826 [SPARK-32529][CORE] Fix Historyserver log scan aborted by application status change
# What changes were proposed in this pull request?
This PR adds a `FileNotFoundException` try catch block while adding a new entry to history server application listing to skip the non-existing path.

### Why are the changes needed?
If there are a large number (>100k) of applications log dir, listing the log dir will take a few seconds. After getting the path list some applications might have finished already, and the filename will change from `foo.inprogress` to `foo`.

It leads to a problem when adding an entry to the listing, querying file status like `fileSizeForLastIndex` will throw out a `FileNotFoundException` exception if the application was finished. And the exception will abort current loop, in a busy cluster, it will make history server couldn't list and load any application log.

```
20/08/03 15:17:23 ERROR FsHistoryProvider: Exception in checking for event log updates
 java.io.FileNotFoundException: File does not exist: hdfs://xx/logs/spark/application_11111111111111.lz4.inprogress
 at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1527)
 at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1520)
 at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
 at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1520)
 at org.apache.spark.deploy.history.SingleFileEventLogFileReader.status$lzycompute(EventLogFileReaders.scala:170)
```

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

### How was this patch tested?
1. setup another script keeps changing the filename of applications under history log dir
2. launch the history server
3. check whether the `File does not exist` error log was gone.

Closes #29350 from yanxiaole/SPARK-32529.

Authored-by: Yan Xiaole <xiaole.yan@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-05 10:57:11 -07:00
Terry Kim 171b7d5d71 [SPARK-23431][CORE] Expose stage level peak executor metrics via REST API
### What changes were proposed in this pull request?

Note that this PR is forked from #23340 originally written by edwinalu.

This PR proposes to expose the peak executor metrics at the stage level via the REST APIs:
* `/applications/<application_id>/stages/`: peak values of executor metrics for **each stage**
* `/applications/<application_id>/stages/<stage_id>/< stage_attempt_id >`: peak values of executor metrics for **each executor** for the stage, followed by peak values of executor metrics for the stage

### Why are the changes needed?

The stage level peak executor metrics can help better understand your application's resource utilization.

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

1. For the `/applications/<application_id>/stages/` API, you will see the following new info for **each stage**:
```JSON
  "peakExecutorMetrics" : {
    "JVMHeapMemory" : 213367864,
    "JVMOffHeapMemory" : 189011656,
    "OnHeapExecutionMemory" : 0,
    "OffHeapExecutionMemory" : 0,
    "OnHeapStorageMemory" : 2133349,
    "OffHeapStorageMemory" : 0,
    "OnHeapUnifiedMemory" : 2133349,
    "OffHeapUnifiedMemory" : 0,
    "DirectPoolMemory" : 282024,
    "MappedPoolMemory" : 0,
    "ProcessTreeJVMVMemory" : 0,
    "ProcessTreeJVMRSSMemory" : 0,
    "ProcessTreePythonVMemory" : 0,
    "ProcessTreePythonRSSMemory" : 0,
    "ProcessTreeOtherVMemory" : 0,
    "ProcessTreeOtherRSSMemory" : 0,
    "MinorGCCount" : 13,
    "MinorGCTime" : 115,
    "MajorGCCount" : 4,
    "MajorGCTime" : 339
  }
```

2. For the `/applications/<application_id>/stages/<stage_id>/<stage_attempt_id>` API, you will see the following new info for **each executor** under `executorSummary`:
```JSON
  "peakMemoryMetrics" : {
    "JVMHeapMemory" : 0,
    "JVMOffHeapMemory" : 0,
    "OnHeapExecutionMemory" : 0,
    "OffHeapExecutionMemory" : 0,
    "OnHeapStorageMemory" : 0,
    "OffHeapStorageMemory" : 0,
    "OnHeapUnifiedMemory" : 0,
    "OffHeapUnifiedMemory" : 0,
    "DirectPoolMemory" : 0,
    "MappedPoolMemory" : 0,
    "ProcessTreeJVMVMemory" : 0,
    "ProcessTreeJVMRSSMemory" : 0,
    "ProcessTreePythonVMemory" : 0,
    "ProcessTreePythonRSSMemory" : 0,
    "ProcessTreeOtherVMemory" : 0,
    "ProcessTreeOtherRSSMemory" : 0,
    "MinorGCCount" : 0,
    "MinorGCTime" : 0,
    "MajorGCCount" : 0,
    "MajorGCTime" : 0
  }
```
, and the following at the stage level:
```JSON
"peakExecutorMetrics" : {
    "JVMHeapMemory" : 213367864,
    "JVMOffHeapMemory" : 189011656,
    "OnHeapExecutionMemory" : 0,
    "OffHeapExecutionMemory" : 0,
    "OnHeapStorageMemory" : 2133349,
    "OffHeapStorageMemory" : 0,
    "OnHeapUnifiedMemory" : 2133349,
    "OffHeapUnifiedMemory" : 0,
    "DirectPoolMemory" : 282024,
    "MappedPoolMemory" : 0,
    "ProcessTreeJVMVMemory" : 0,
    "ProcessTreeJVMRSSMemory" : 0,
    "ProcessTreePythonVMemory" : 0,
    "ProcessTreePythonRSSMemory" : 0,
    "ProcessTreeOtherVMemory" : 0,
    "ProcessTreeOtherRSSMemory" : 0,
    "MinorGCCount" : 13,
    "MinorGCTime" : 115,
    "MajorGCCount" : 4,
    "MajorGCTime" : 339
  }
```

### How was this patch tested?

Added tests.

Closes #29020 from imback82/metrics.

Lead-authored-by: Terry Kim <yuminkim@gmail.com>
Co-authored-by: edwinalu <edwina.lu@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-08-04 21:11:00 +08:00
Takuya UESHIN 7deb67c28f [SPARK-32160][CORE][PYSPARK][FOLLOWUP] Change the config name to switch allow/disallow SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #29278.
This PR changes the config name to switch allow/disallow `SparkContext` in executors as per the comment https://github.com/apache/spark/pull/29278#pullrequestreview-460256338.

### Why are the changes needed?

The config name `spark.executor.allowSparkContext` is more reasonable.

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

Yes, the config name is changed.

### How was this patch tested?

Updated tests.

Closes #29340 from ueshin/issues/SPARK-32160/change_config_name.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-04 12:45:06 +09:00
Takeshi Yamamuro c6109ba918 [SPARK-32257][SQL] Reports explicit errors for invalid usage of SET/RESET command
### What changes were proposed in this pull request?

This PR modified the parser code to handle invalid usages of a SET/RESET command.
For example;
```
SET spark.sql.ansi.enabled true
```
The above SQL command does not change the configuration value and it just tries to display the value of the configuration
`spark.sql.ansi.enabled true`. This PR disallows using special characters including spaces in the configuration name and reports a user-friendly error instead. In the error message, it tells users a workaround to use quotes or a string literal if they still needs to specify a configuration with them. 

Before this PR:
```
scala> sql("SET spark.sql.ansi.enabled true").show(1, -1)
+---------------------------+-----------+
|key                        |value      |
+---------------------------+-----------+
|spark.sql.ansi.enabled true|<undefined>|
+---------------------------+-----------+
```

After this PR:
```
scala> sql("SET spark.sql.ansi.enabled true")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
SET spark.sql.ansi.enabled true
^^^
```

### Why are the changes needed?

For better user-friendly errors.

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

No.

### How was this patch tested?

Added tests in `SparkSqlParserSuite`.

Closes #29146 from maropu/SPARK-32257.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 13:00:07 +00:00
Gengliang Wang 71aea02e9f [SPARK-32467][UI] Avoid encoding URL twice on https redirect
### What changes were proposed in this pull request?

When https is enabled for Spark UI, an HTTP request will be redirected as an encoded HTTPS URL: https://github.com/apache/spark/pull/10238/files#diff-f79a5ead735b3d0b34b6b94486918e1cR312

When we create the redirect url, we will call getRequestURI and getQueryString. Both two methods may return an encoded string. However, we pass them directly to the following URI constructor
```
URI(String scheme, String authority, String path, String query, String fragment)
```
As this URI constructor assumes both path and query parameters are decoded strings, it will encode them again. This makes the redirect URL encoded twice.

This problem is on stage page with HTTPS enabled. The URL of "/taskTable" contains query parameter `order%5B0%5D%5Bcolumn%5D`. After encoded it becomes  `order%255B0%255D%255Bcolumn%255D` and it will be decoded as `order%5B0%5D%5Bcolumn%5D` instead of `order[0][dir]`.  When the parameter `order[0][dir]` is missing, there will be an excetpion from:
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala#L176
and the stage page fail to load.

To fix the problem, we can try decoding the query parameters before encoding it. This is to make sure we encode the URL

### Why are the changes needed?

Fix a UI issue when HTTPS is enabled

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

No

### How was this patch tested?

A new Unit test + manually test on a cluster

Closes #29271 from gengliangwang/urlEncode.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-08-01 13:09:26 +08:00
Venkata krishnan Sowrirajan 4eaf3a0a23 [SPARK-31418][CORE][FOLLOW-UP][MINOR] Fix log messages to print stage id instead of the object name
### What changes were proposed in this pull request?
Just few log lines fixes which are logging the object name instead of the stage IDs

### Why are the changes needed?
This would make it easier later for debugging.

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

### How was this patch tested?
Just log messages. Existing tests should be enough

Closes #29279 from venkata91/SPARK-31418-follow-up.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 22:12:24 +09:00
Takuya UESHIN 8014b0b5d6 [SPARK-32160][CORE][PYSPARK] Add a config to switch allow/disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #28986.
This PR adds a config to switch allow/disallow to create `SparkContext` in executors.

- `spark.driver.allowSparkContextInExecutors`

### Why are the changes needed?

Some users or libraries actually create `SparkContext` in executors.
We shouldn't break their workloads.

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

Yes, users will be able to create `SparkContext` in executors with the config enabled.

### How was this patch tested?

More tests are added.

Closes #29278 from ueshin/issues/SPARK-32160/add_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 17:28:35 +09:00
Kousuke Saruta 9d7b1d935f [SPARK-32175][SPARK-32175][FOLLOWUP] Remove flaky test added in
### What changes were proposed in this pull request?

This PR removes a test added in SPARK-32175(#29002).

### Why are the changes needed?

That test is flaky. It can be mitigated by increasing the timeout but it would rather be simpler to remove the test.
See also the [discussion](https://github.com/apache/spark/pull/29002#issuecomment-666746857).

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

No.

Closes #29314 from sarutak/remove-flaky-test.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-07-31 10:37:05 +09:00
Devesh Agrawal 6032c5b032 [SPARK-32417] Fix flakyness of BlockManagerDecommissionIntegrationSuite
### What changes were proposed in this pull request?

This test tries to fix the flakyness of BlockManagerDecommissionIntegrationSuite.

### Description of the problem

Make the block manager decommissioning test be less flaky

An interesting failure happens when migrateDuring = true (and persist or shuffle is true):
- We schedule the job with tasks on executors 0, 1, 2.
- We wait 300 ms and decommission executor 0.
- If the task is not yet done on executor 0, it will now fail because
   the block manager won't be able to save the block. This condition is
   easy to trigger on a loaded machine where the github checks run.
- The task with retry on a different executor (1 or 2) and its shuffle
   blocks will land there.
- No actual block migration happens here because the decommissioned
   executor technically failed before it could even produce a block.

To remove the above race, this change replaces the fixed wait for 300 ms to wait for an actual task to succeed. When a task has succeeded, we know its blocks would have been written for sure and thus its executor would certainly be forced to migrate those blocks when it is decommissioned.

The change always decommissions an executor on which a real task finished successfully instead of picking the first executor. Because the system may choose to schedule nothing on the first executor and instead run the two tasks on one executor.

### Why are the changes needed?

I have had bad luck with BlockManagerDecommissionIntegrationSuite and it has failed several times on my PRs. So fixing it.

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

No, unit test only change.

### How was this patch tested?

Github checks. Ran this test 100 times, 10 at a time in parallel in a script.

Closes #29226 from agrawaldevesh/block-manager-decom-flaky.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-30 12:00:19 -07:00
Devesh Agrawal 366a178933 [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning
### What changes were proposed in this pull request?

This PR reduces the prospect of a job loss during decommissioning. It
fixes two holes in the current decommissioning framework:

- (a) Loss of decommissioned executors is not treated as a job failure:
We know that the decommissioned executor would be dying soon, so its death is
clearly not caused by the application.

- (b) Shuffle files on the decommissioned host are cleared when the
first fetch failure is detected from a decommissioned host: This is a
bit tricky in terms of when to clear the shuffle state ? Ideally you
want to clear it the millisecond before the shuffle service on the node
dies (or the executor dies when there is no external shuffle service) --
too soon and it could lead to some wastage and too late would lead to
fetch failures.

  The approach here is to do this clearing when the very first fetch
failure is observed on the decommissioned block manager, without waiting for
other blocks to also signal a failure.

### Why are the changes needed?

Without them decommissioning a lot of executors at a time leads to job failures.

### Code overview

The task scheduler tracks the executors that were decommissioned along with their
`ExecutorDecommissionInfo`. This information is used by: (a) For handling a `ExecutorProcessLost` error, or (b) by the `DAGScheduler` when handling a fetch failure.

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

No

### How was this patch tested?

Added a new unit test `DecommissionWorkerSuite` to test the new behavior by exercising the Master-Worker decommissioning. I chose to add a new test since the setup logic was quite different from the existing `WorkerDecommissionSuite`. I am open to changing the name of the newly added test suite :-)

### Questions for reviewers
- Should I add a feature flag to guard these two behaviors ? They seem safe to me that they should only get triggered by decommissioning, but you never know :-).

Closes #29014 from agrawaldevesh/decom_harden.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-30 11:58:11 -07:00
Dongjoon Hyun 7cf3b54a2a [SPARK-32489][CORE] Pass core module UTs in Scala 2.13
### What changes were proposed in this pull request?

So far, we fixed many stuffs in `core` module. This PR fixes the remaining UT failures in Scala 2.13.

- `OneApplicationResource.environmentInfo` will return a deterministic result for `sparkProperties`, `hadoopProperties`, `systemProperties`, and `classpathEntries`.
- `SubmitRestProtocolSuite` has Scala 2.13 answer in addition to the existing Scala 2.12 answer, and uses the expected answer based on the Scala runtime version.

### Why are the changes needed?

To support Scala 2.13.

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

Yes, `environmentInfo` is changed, but this fixes the indeterministic behavior.

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action
- Scala 2.13: Do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13
```

**BEFORE**
```
Tests: succeeded 2612, failed 3, canceled 1, ignored 8, pending 0
*** 3 TESTS FAILED ***
```

**AFTER**
```
Tests: succeeded 2615, failed 0, canceled 1, ignored 8, pending 0
All tests passed.
```

Closes #29298 from dongjoon-hyun/SPARK-32489.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-30 10:59:26 -07:00
Dongjoon Hyun 163867435a [SPARK-32487][CORE] Remove j.w.r.NotFoundException from import in [Stages|OneApplication]Resource
### What changes were proposed in this pull request?

This PR aims to remove `java.ws.rs.NotFoundException` from two problematic `import` statements. All the other use cases are correct.

### Why are the changes needed?

In `StagesResource` and `OneApplicationResource`, there exist two `NotFoundException`s.
- javax.ws.rs.NotFoundException
- org.apache.spark.status.api.v1.NotFoundException

To use `org.apache.spark.status.api.v1.NotFoundException` correctly, we should not import `java.ws.rs.NotFoundException`. This causes UT failures in Scala 2.13 environment.

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

No.

### How was this patch tested?

- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Do the following manually.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.HistoryServerSuite
```

**BEFORE**
```
*** 4 TESTS FAILED ***
```

**AFTER**
```
*** 1 TEST FAILED ***
```

Closes #29293 from dongjoon-hyun/SPARK-32487.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 17:57:46 -07:00
Dongjoon Hyun 9dc0237851 [SPARK-32476][CORE] ResourceAllocator.availableAddrs should be deterministic
### What changes were proposed in this pull request?

This PR aims to make `ResourceAllocator.availableAddrs` deterministic.

### Why are the changes needed?

Currently, this function returns indeterministically due to the underlying `HashMap`. So, the test case itself is creating a list `[0, 1, 2]` initially, but ends up with comparing `[2, 1, 0]`.

Not only this happens in the 3.0.0, but also this causes UT failures on Scala 2.13 environment.

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

Yes, but this fixes the in-deterministic behavior.

### How was this patch tested?

- Scala 2.12: This should pass the UT with the modified test case.
- Scala 2.13: This can be tested like the following (at least `JsonProtocolSuite`)

```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.JsonProtocolSuite
```

**BEFORE**
```
*** 2 TESTS FAILED ***
```

**AFTER**
```
All tests passed.
```

Closes #29281 from dongjoon-hyun/SPARK-32476.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 10:47:32 -07:00
Dongjoon Hyun 5eab8d27e6 [SPARK-32477][CORE] JsonProtocol.accumulablesToJson should be deterministic
### What changes were proposed in this pull request?

This PR aims to make `JsonProtocol.accumulablesToJson` deterministic.

### Why are the changes needed?

Currently, `JsonProtocol.accumulablesToJson` is indeterministic. So, `JsonProtocolSuite` itself is also using mixed test cases in terms of `"Accumulables": [ ... ]`.

Not only this is indeterministic, but also this causes a UT failure in `JsonProtocolSuite` in Scala 2.13.

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

Yes. However, this is a fix on indeterministic behavior.

### How was this patch tested?

- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.JsonProtocolSuite
```

**BEFORE**
```
*** 1 TEST FAILED ***
```

**AFTER**
```
All tests passed.
```

Closes #29282 from dongjoon-hyun/SPARK-32477.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 07:48:23 -07:00
Kousuke Saruta 9be088357e [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread
### What changes were proposed in this pull request?

This PR changes the order between initialization for ExecutorPlugin and starting heartbeat thread in Executor.

### Why are the changes needed?

In the current master, heartbeat thread in a executor starts after plugin initialization so if the initialization takes long time, heartbeat is not sent to driver and the executor will be removed from cluster.

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

Yes. Plugins for executors will be allowed to take long time for initialization.

### How was this patch tested?

New testcase.

Closes #29002 from sarutak/fix-heartbeat-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-29 08:44:56 -05:00
Dongjoon Hyun 77987a222c [SPARK-32473][CORE][TESTS] Use === instead IndexSeqView
### What changes were proposed in this pull request?

This PR aims to fix `SorterSuite` and `RadixSortSuite` in Scala 2.13 by using `===` instead of `IndexSeqView`.
```
$ git grep "\.view =="
core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala:    assert(data0.view === data1.view)
core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala:    assert(data0.view === data2.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(ref.view == result.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(res1.view == res2.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(ref.view == result.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(res1.view == res2.view)
```

### Why are the changes needed?

Scala 2.13 reimplements `IndexSeqView` and the behavior is different.
- https://docs.scala-lang.org/overviews/core/collections-migration-213.html

**Scala 2.12**
```scala
Welcome to Scala 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> Seq(1,2,3).toArray.view == Seq(1,2,3).toArray.view
res0: Boolean = true
```

**Scala 2.13**
```scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> Seq(1,2,3).toArray.view == Seq(1,2,3).toArray.view
val res0: Boolean = false
```

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

No. This is a test-only fix.

### How was this patch tested?

- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Manually test the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.collection.unsafe.sort.RadixSortSuite
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.collection.SorterSuite
```

**BEFORE**
```
Tests: succeeded 9, failed 36, canceled 0, ignored 0, pending 0
*** 36 TESTS FAILED ***
Tests: succeeded 3, failed 1, canceled 0, ignored 2, pending 0
*** 1 TEST FAILED ***
```

**AFTER**
```
Tests: succeeded 45, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
Tests: succeeded 4, failed 0, canceled 0, ignored 2, pending 0
All tests passed.
```

Closes #29280 from dongjoon-hyun/SPARK-32473.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-28 22:31:39 -07:00
LantaoJin 26e6574d58 [SPARK-32283][CORE] Kryo should support multiple user registrators
### What changes were proposed in this pull request?
`spark.kryo.registrator` in 3.0 has a regression problem. From [SPARK-12080](https://issues.apache.org/jira/browse/SPARK-12080), it supports multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it donsn't work in 3.0. Fix it by `toSequence` in `Kryo.scala`

### Why are the changes needed?
In previous Spark version (2.x), it supported multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it doesn't work in 3.0. It's should be a regression.

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

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

Closes #29123 from LantaoJin/SPARK-32283.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:58:03 +00:00
HyukjinKwon c1140661bf [SPARK-32443][CORE] Use POSIX-compatible command -v in testCommandAvailable
### What changes were proposed in this pull request?

This PR aims to use `command -v` in non-Window operating systems instead of executing the given command.

### Why are the changes needed?

1. `command` is POSIX-compatible
    - **POSIX.1-2017**:  https://pubs.opengroup.org/onlinepubs/9699919799/utilities/command.html
2. `command` is faster and safer than the direct execution
    - `command` doesn't invoke another process.
```scala
scala> sys.process.Process("ls").run().exitValue()
LICENSE
NOTICE
bin
doc
lib
man
res1: Int = 0
```

3. The existing way behaves inconsistently.
    - `rm` cannot be checked.

**AS-IS**
```scala
scala> sys.process.Process("rm").run().exitValue()
usage: rm [-f | -i] [-dPRrvW] file ...
       unlink file
res0: Int = 64
```

**TO-BE**
```
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.
scala> sys.process.Process(Seq("sh", "-c", s"command -v ls")).run().exitValue()
/bin/ls
val res1: Int = 0
```

4. The existing logic is already broken in Scala 2.13 environment because it hangs like the following.
```scala
$ bin/scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> sys.process.Process("cat").run().exitValue() // hang here.
```

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

No. Although this is inside `main` source directory, this is used for testing purpose.

```
$ git grep testCommandAvailable | grep -v 'def testCommandAvailable'
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("wc"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(!TestUtils.testCommandAvailable("some_nonexistent_command"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:  private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec)
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:    if (TestUtils.testCommandAvailable(pythonExec)) {
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("python"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("echo | sed"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
```

### How was this patch tested?

- **Scala 2.12**: Pass the Jenkins with the existing tests and one modified test.
- **Scala 2.13**: Do the following manually. It should pass instead of `hang`.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.PipedRDDSuite
...
Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29241 from dongjoon-hyun/SPARK-32443.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-27 12:02:43 -07:00
Warren Zhu 998086c9a1 [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory
### What changes were proposed in this pull request?
Support set off heap memory in `ExecutorResourceRequests`

### Why are the changes needed?
Support stage level scheduling

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

### How was this patch tested?
Added UT in `ResourceProfileSuite` and `DAGSchedulerSuite`

Closes #28972 from warrenzhu25/30794.

Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-27 08:16:13 -05:00
Dongjoon Hyun 7e0c5b3b53 [SPARK-32442][CORE][TESTS] Fix TaskSetManagerSuite by hiding o.a.s.FakeSchedulerBackend
### What changes were proposed in this pull request?

There exists two `FakeSchedulerBackend` classes.
```
$ git grep "class FakeSchedulerBackend"
core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala:private class FakeSchedulerBackend(
core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala:class FakeSchedulerBackend extends SchedulerBackend {
```

This PR aims to hide the following at `TaskSetManagerSuite`.
```scala
import org.apache.spark.{FakeSchedulerBackend => _, _}
```

### Why are the changes needed?

Although `TaskSetManagerSuite` is inside `org.apache.spark.scheduler` package, `import org.apache.spark._` makes Scala 2.13 confused and causes 4 UT failures.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.TaskSetManagerSuite
...
Tests: succeeded 48, failed 4, canceled 0, ignored 0, pending 0
*** 4 TESTS FAILED ***
```

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

No.

### How was this patch tested?

- **Scala 2.12**: Pass the Jenkins or GitHub Action
- **Scala 2.13**: Pass the following manually.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.TaskSetManagerSuite
...
Tests: succeeded 52, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29240 from dongjoon-hyun/SPARK-32442.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-26 07:54:30 -07:00
Dongjoon Hyun 147022a5c6 [SPARK-32440][CORE][TESTS] Make BlockManagerSuite robust from Scala object size difference
### What changes were proposed in this pull request?

This PR aims to increase the memory parameter in `BlockManagerSuite`'s worker decommission test cases.

### Why are the changes needed?

Scala 2.13 generates different Java objects and this affects Spark's `SizeEstimator/SizeTracker/SizeTrackingVector`. This causes UT failures like the following. If we decrease the values, those test cases fails in Scala 2.12, too.

```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.storage.BlockManagerSuite
...
- test decommission block manager should not be part of peers *** FAILED ***
  0 did not equal 2 (BlockManagerSuite.scala:1869)
- test decommissionRddCacheBlocks should offload all cached blocks *** FAILED ***
  0 did not equal 2 (BlockManagerSuite.scala:1884)
...
Tests: succeeded 81, failed 2, canceled 0, ignored 0, pending 0
*** 2 TESTS FAILED ***
```

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

No.

### How was this patch tested?

```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.storage.BlockManagerSuite
...
Tests: succeeded 83, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29238 from dongjoon-hyun/SPARK-32440.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 15:54:21 -07:00
Dongjoon Hyun 80e8898158 [SPARK-32438][CORE][TESTS] Use HashMap.withDefaultValue in RDDSuite
### What changes were proposed in this pull request?

Since Scala 2.13, `HashMap` is changed to become a final in the future and `.withDefault` is recommended. This PR aims to use `HashMap.withDefaultValue` instead of overriding manually in the test case.

- https://www.scala-lang.org/api/current/scala/collection/mutable/HashMap.html

```scala
deprecatedInheritance(message =
"HashMap wil be made final; use .withDefault for the common use case of computing a default value",
since = "2.13.0")
```

### Why are the changes needed?

In Scala 2.13, the existing code causes a failure because the default value function doesn't work correctly.

```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.RDDSuite
- aggregate *** FAILED ***
  org.apache.spark.SparkException: Job aborted due to stage failure:
Task 0 in stage 61.0 failed 1 times, most recent failure: Lost task 0.0 in stage 61.0 (TID 198, localhost, executor driver):
java.util.NoSuchElementException: key not found: a
```

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

No. This is a test case change.

### How was this patch tested?

1. **Scala 2.12:** Pass the Jenkins or GitHub with the existing tests.
2. **Scala 2.13**: Manually do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.RDDSuite
...
Tests: succeeded 72, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29235 from dongjoon-hyun/SPARK-32438.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 10:52:55 -07:00
Dongjoon Hyun f9f18673dc [SPARK-32436][CORE] Initialize numNonEmptyBlocks in HighlyCompressedMapStatus.readExternal
### What changes were proposed in this pull request?

This PR aims to initialize `numNonEmptyBlocks` in `HighlyCompressedMapStatus.readExternal`.

In Scala 2.12, this is initialized to `-1` via the following.
```scala
protected def this() = this(null, -1, null, -1, null, -1)  // For deserialization only
```

### Why are the changes needed?

In Scala 2.13, this causes several UT failures because `HighlyCompressedMapStatus.readExternal` doesn't initialize this field. The following is one example.

- org.apache.spark.scheduler.MapStatusSuite
```
MapStatusSuite:
- compressSize
- decompressSize
*** RUN ABORTED ***
  java.lang.NoSuchFieldError: numNonEmptyBlocks
  at org.apache.spark.scheduler.HighlyCompressedMapStatus.<init>(MapStatus.scala:181)
  at org.apache.spark.scheduler.HighlyCompressedMapStatus$.apply(MapStatus.scala:281)
  at org.apache.spark.scheduler.MapStatus$.apply(MapStatus.scala:73)
  at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$8(MapStatusSuite.scala:64)
  at scala.runtime.java8.JFunction1$mcVD$sp.apply(JFunction1$mcVD$sp.scala:18)
  at scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$7(MapStatusSuite.scala:61)
  at scala.runtime.java8.JFunction1$mcVJ$sp.apply(JFunction1$mcVJ$sp.scala:18)
  at scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.scheduler.MapStatusSuite.$anonfun$new$6(MapStatusSuite.scala:60)
  ...
```

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

No. This is a private class.

### How was this patch tested?

1. Pass the GitHub Action or Jenkins with the existing tests.
2. Test with Scala-2.13 with `MapStatusSuite`.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.scheduler.MapStatusSuite
...
MapStatusSuite:
- compressSize
- decompressSize
- MapStatus should never report non-empty blocks' sizes as 0
- large tasks should use org.apache.spark.scheduler.HighlyCompressedMapStatus
- HighlyCompressedMapStatus: estimated size should be the average non-empty block size
- SPARK-22540: ensure HighlyCompressedMapStatus calculates correct avgSize
- RoaringBitmap: runOptimize succeeded
- RoaringBitmap: runOptimize failed
- Blocks which are bigger than SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be underestimated.
- SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE
Run completed in 7 seconds, 971 milliseconds.
Total number of tests run: 10
Suites: completed 2, aborted 0
Tests: succeeded 10, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29231 from dongjoon-hyun/SPARK-32436.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 10:16:01 -07:00
Dongjoon Hyun f642234d85 [SPARK-32437][CORE] Improve MapStatus deserialization speed with RoaringBitmap 0.9.0
### What changes were proposed in this pull request?

This PR aims to speed up `MapStatus` deserialization by 5~18% with the latest RoaringBitmap `0.9.0` and new APIs. Note that we focus on `deserialization` time because `serialization` occurs once while `deserialization` occurs many times.

### Why are the changes needed?

The current version is too old. We had better upgrade it to get the performance improvement and bug fixes.
Although `MapStatusesSerDeserBenchmark` is synthetic, the benchmark result is updated with this patch.

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

No.

### How was this patch tested?

Pass the Jenkins or GitHub Action.

Closes #29233 from dongjoon-hyun/SPARK-ROAR.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 08:07:28 -07:00
Gabor Somogyi b890fdc8df [SPARK-32387][SS] Extract UninterruptibleThread runner logic from KafkaOffsetReader
### What changes were proposed in this pull request?
`UninterruptibleThread` running functionality is baked into `KafkaOffsetReader` which can be extracted into a class. The main intention is to simplify `KafkaOffsetReader` in order to make easier to solve SPARK-32032. In this PR I've made this extraction without functionality change.

### Why are the changes needed?
`UninterruptibleThread` running functionality is baked into `KafkaOffsetReader`.

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

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

Closes #29187 from gaborgsomogyi/SPARK-32387.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-24 11:41:42 -07:00
Thomas Graves e6ef27be52 [SPARK-32287][TESTS] Flaky Test: ExecutorAllocationManagerSuite.add executors default profile
### What changes were proposed in this pull request?

I wasn't able to reproduce the failure but the best I can tell is that the allocation manager timer triggers and call doRequest. The timeout is 10s so try to increase that to 30seconds.

### Why are the changes needed?

test failure

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

no

### How was this patch tested?

unit test

Closes #29225 from tgravescs/SPARK-32287.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-24 11:12:28 -07:00
Sean Owen be2eca22e9 [SPARK-32398][TESTS][CORE][STREAMING][SQL][ML] Update to scalatest 3.2.0 for Scala 2.13.3+
### What changes were proposed in this pull request?

Updates to scalatest 3.2.0. Though it looks large, it is 99% changes to the new location of scalatest classes.

### Why are the changes needed?

3.2.0+ has a fix that is required for Scala 2.13.3+ compatibility.

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

No, only affects tests.

### How was this patch tested?

Existing tests.

Closes #29196 from srowen/SPARK-32398.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 16:20:17 -07:00
Venkata krishnan Sowrirajan e7fb67cd88 [SPARK-31418][SCHEDULER] Request more executors in case of dynamic allocation is enabled and a task becomes unschedulable due to spark's blacklisting feature
### What changes were proposed in this pull request?
In this change, when dynamic allocation is enabled instead of aborting immediately when there is an unschedulable taskset due to blacklisting, pass an event saying `SparkListenerUnschedulableTaskSetAdded` which will be handled by `ExecutorAllocationManager` and request more executors needed to schedule the unschedulable blacklisted tasks. Once the event is sent, we start the abortTimer similar to [SPARK-22148][SPARK-15815] to abort in the case when no new executors launched either due to max executors reached or cluster manager is out of capacity.

### Why are the changes needed?
This is an improvement. In the case when dynamic allocation is enabled, this would request more executors to schedule the unschedulable tasks instead of aborting the stage without even retrying upto spark.task.maxFailures times (in some cases not retrying at all). This is a potential issue with respect to Spark's Fault tolerance.

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

### How was this patch tested?
Added unit tests both in ExecutorAllocationManagerSuite and TaskSchedulerImplSuite

Closes #28287 from venkata91/SPARK-31418.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-23 12:33:22 -05:00
Devesh Agrawal f8d29d371c [SPARK-32217] Plumb whether a worker would also be decommissioned along with executor
### What changes were proposed in this pull request?

This PR is a giant plumbing PR that plumbs an `ExecutorDecommissionInfo` along
with the DecommissionExecutor message.

### Why are the changes needed?

The primary motivation is to know whether a decommissioned executor
would also be loosing shuffle files -- and thus it is important to know
whether the host would also be decommissioned.

In the absence of this PR, the existing code assumes that decommissioning an executor does not loose the whole host with it, and thus does not clear the shuffle state if external shuffle service is enabled. While this may hold in some cases (like K8s decommissioning an executor pod, or YARN container preemption), it does not hold in others like when the cluster is managed by a Standalone Scheduler (Master). This is similar to the existing `workerLost` field in the `ExecutorProcessLost` message.

In the future, this `ExecutorDecommissionInfo` can be embellished for
knowing how long the executor has to live for scenarios like Cloud spot
kills (or Yarn preemption) and the like.

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

### How was this patch tested?
Tweaked an existing unit test in `AppClientSuite`

Closes #29032 from agrawaldevesh/plumb_decom_info.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-22 21:04:06 -07:00
Wing Yew Poon e8c06af7d1 [SPARK-32003][CORE] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost
### What changes were proposed in this pull request?

If an executor is lost, the `DAGScheduler` handles the executor loss by removing the executor but does not unregister its outputs if the external shuffle service is used. However, if the node on which the executor runs is lost, the shuffle service may not be able to serve the shuffle files.
In such a case, when fetches from the executor's outputs fail in the same stage, the `DAGScheduler` again removes the executor and by right, should unregister its outputs. It doesn't because the epoch used to track the executor failure has not increased.

We track the epoch for failed executors that result in lost file output separately, so we can unregister the outputs in this scenario. The idea to track a second epoch is due to Attila Zsolt Piros.

### Why are the changes needed?

Without the changes, the loss of a node could require two stage attempts to recover instead of one.

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

No.

### How was this patch tested?

New unit test. This test fails without the change and passes with it.

Closes #28848 from wypoon/SPARK-32003.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2020-07-22 09:53:16 -05:00
Max Gekk feca9edbdd [MINOR][SQL][TESTS] Create tables once in JDBC tests
### What changes were proposed in this pull request?
In PR, I propose to create input tables once before executing tests in `JDBCSuite` and `JdbcRDDSuite`. Currently, the table are created before every test in the test suites.

### Why are the changes needed?
This speed up the test suites up 30-40%.

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

### How was this patch tested?
Run the modified test suites

Closes #29176 from MaxGekk/jdbc-suite-before-all.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 08:32:01 +00:00
yi.wu b4a9606890 [SPARK-31922][CORE] logDebug "RpcEnv already stopped" error on LocalSparkCluster shutdown
### What changes were proposed in this pull request?

Catch the `RpcEnvStoppedException` and log debug it when stop is called for a `LocalSparkCluster`.

This PR also contains two small changes to fix the potential issues.

### Why are the changes needed?

Currently, there's always "RpcEnv already stopped" error if we exit spark-shell with local-cluster mode:

```
20/06/07 14:54:18 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() for one-way message.
org.apache.spark.rpc.RpcEnvStoppedException: RpcEnv already stopped.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:167)
        at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:150)
        at org.apache.spark.rpc.netty.NettyRpcHandler.receive(NettyRpcEnv.scala:691)
        at org.apache.spark.network.server.TransportRequestHandler.processOneWayMessage(TransportRequestHandler.java:253)
        at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:111)
        at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:140)
        at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:53)
        at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
        at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
        at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
        at org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:102)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
        at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
        at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
        at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163)
        at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:714)
        at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:650)
        at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:576)
        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)

```

When we call stop on `StandaloneSchedulerBackend`, the backend will firstly send `UnregisterApplication` to `Master` and then call stop on `LocalSparkCluster` immediately. On the other side, `Master` will send messages to `Worker` when it receives `UnregisterApplication`.  However, the rpcEnv of the `Worker` has been already stoped by the backend. Therefore, the error message shows when the `Worker` tries to handle the messages.

It's only an error on shutdown, users would not like to care about it. So we could hide it in debug log and this is also what we've done previously in #18547.

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

Yes, users will not see the error message after this PR.

### How was this patch tested?

Tested manually.

Closes #28746 from Ngone51/fix-spark-31922.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-21 23:16:38 -07:00
Baohe Zhang 7b9d7551a6 [SPARK-32350][CORE] Add batch-write on LevelDB to improve performance of HybridStore
### What changes were proposed in this pull request?
The idea is to improve the performance of HybridStore by adding batch write support to LevelDB. #28412  introduces HybridStore. HybridStore will write data to InMemoryStore at first and use a background thread to dump data to LevelDB once the writing to InMemoryStore is completed. In the comments section of #28412 , mridulm mentioned using batch writing can improve the performance of this dumping process and he wrote the code of writeAll().

### Why are the changes needed?
I did the comparison of the HybridStore switching time between one-by-one write and batch write on an HDD disk. When the disk is free, the batch-write has around 25% improvement, and when the disk is 100% busy, the batch-write has 7x - 10x improvement.

when the disk is at 0% utilization:
| log size, jobs and tasks per job   | original switching time, with write() | switching time with writeAll() |
| ---------------------------------- | ------------------------------------- | ------------------------------ |
| 133m, 400 jobs, 100 tasks per job  | 16s                                   | 13s                            |
| 265m, 400 jobs, 200 tasks per job  | 30s                                   | 23s                            |
| 1.3g, 1000 jobs, 400 tasks per job | 136s                                  | 108s                           |

when the disk is at 100% utilization:
| log size, jobs and tasks per job  | original switching time, with write() | switching time with writeAll() |
| --------------------------------- | ------------------------------------- | ------------------------------ |
| 133m, 400 jobs, 100 tasks per job | 116s                                  | 17s                            |
| 265m, 400 jobs, 200 tasks per job | 251s                                  | 26s                            |

I also ran some write related benchmarking tests on LevelDBBenchmark.java and measured the total time of writing 1024 objects. The tests were conducted when the disk is at 0% utilization.

| Benchmark test           | with write(), ms | with writeAll(), ms |
| ------------------------ | ---------------- | ------------------- |
| randomUpdatesIndexed     | 213.06           | 157.356             |
| randomUpdatesNoIndex     | 57.869           | 35.439              |
| randomWritesIndexed      | 298.854          | 229.274             |
| randomWritesNoIndex      | 66.764           | 38.361              |
| sequentialUpdatesIndexed | 87.019           | 56.219              |
| sequentialUpdatesNoIndex | 61.851           | 41.942              |
| sequentialWritesIndexed  | 94.044           | 56.534              |
| sequentialWritesNoIndex  | 118.345          | 66.483              |

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

### How was this patch tested?
Manually tested.

Closes #29149 from baohe-zhang/SPARK-32350.

Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-22 13:27:34 +09:00
Holden Karau a4ca355af8 [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown
### What is changed?

This pull request adds the ability to migrate shuffle files during Spark's decommissioning. The design document associated with this change is at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE .

To allow this change the `MapOutputTracker` has been extended to allow the location of shuffle files to be updated with `updateMapOutput`. When a shuffle block is put, a block update message will be sent which triggers the `updateMapOutput`.

Instead of rejecting remote puts of shuffle blocks `BlockManager` delegates the storage of shuffle blocks to it's shufflemanager's resolver (if supported). A new, experimental, trait is added for shuffle resolvers to indicate they handle remote putting of blocks.

The existing block migration code is moved out into a separate file, and a producer/consumer model is introduced for migrating shuffle files from the host as quickly as possible while not overwhelming other executors.

### Why are the changes needed?

Recomputting shuffle blocks can be expensive, we should take advantage of our decommissioning time to migrate these blocks.

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

This PR introduces two new configs parameters, `spark.storage.decommission.shuffleBlocks.enabled` & `spark.storage.decommission.rddBlocks.enabled` that control which blocks should be migrated during storage decommissioning.

### How was this patch tested?

New unit test & expansion of the Spark on K8s decom test to assert that decommisioning with shuffle block migration means that the results are not recomputed even when the original executor is terminated.

This PR is a cleaned-up version of the previous WIP PR I made https://github.com/apache/spark/pull/28331 (thanks to attilapiros for his very helpful reviewing on it :)).

Closes #28708 from holdenk/SPARK-20629-copy-shuffle-data-when-nodes-are-being-shutdown-cleaned-up.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Co-authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Co-authored-by: Attila Zsolt Piros <attilazsoltpiros@apiros-mbp16.lan>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-19 21:33:13 -07:00
Prakhar Jain 0678afe393 [SPARK-21040][CORE] Speculate tasks which are running on decommission executors
### What changes were proposed in this pull request?
This PR adds functionality to consider the running tasks on decommission executors based on some config.
In spark-on-cloud , we sometimes already know that an executor won't be alive for more than fix amount of time. Ex- In AWS Spot nodes, once we get the notification, we know that a node will be gone in 120 seconds.
So if the running tasks on the decommissioning executors may run beyond currentTime+120 seconds, then they are candidate for speculation.

### Why are the changes needed?
Currently when an executor is decommission, we stop scheduling new tasks on those executors but the already running tasks keeps on running on them. Based on the cloud, we might know beforehand that an executor won't be alive for more than a preconfigured time. Different cloud providers gives different timeouts before they take away the nodes. For Ex- In case of AWS spot nodes, an executor won't be alive for more than 120 seconds. We can utilize this information in cloud environments and take better decisions about speculating the already running tasks on decommission executors.

### Does this PR introduce _any_ user-facing change?
Yes. This PR adds a new config "spark.executor.decommission.killInterval" which they can explicitly set based on the cloud environment where they are running.

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

Closes #28619 from prakharjain09/SPARK-21040-speculate-decommission-exec-tasks.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-17 16:11:02 -07:00
Devesh Agrawal ffdbbae1d4 [SPARK-32215] Expose a (protected) /workers/kill endpoint on the MasterWebUI
### What changes were proposed in this pull request?

This PR allows an external agent to inform the Master that certain hosts
are being decommissioned.

### Why are the changes needed?

The current decommissioning is triggered by the Worker getting getting a SIGPWR
(out of band possibly by some cleanup hook), which then informs the Master
about it. This approach may not be feasible in some environments that cannot
trigger a clean up hook on the Worker. In addition, when a large number of
worker nodes are being decommissioned then the master will get a flood of
messages.

So we add a new post endpoint `/workers/kill` on the MasterWebUI that allows an
external agent to inform the master about all the nodes being decommissioned in
bulk. The list of nodes is specified by providing a list of hostnames. All workers on those
hosts will be decommissioned.

This API is merely a new entry point into the existing decommissioning
logic. It does not change how the decommissioning request is handled in
its core.

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

Yes, a new endpoint `/workers/kill` is added to the MasterWebUI. By default only
requests originating from an IP address local to the MasterWebUI are allowed.

### How was this patch tested?

Added unit tests

Closes #29015 from agrawaldevesh/master_decom_endpoint.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-17 06:04:34 +00:00
Warren Zhu db47c6e340 [SPARK-32125][UI] Support get taskList by status in Web UI and SHS Rest API
### What changes were proposed in this pull request?
Support fetching taskList by status as below:
```
/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskList?status=failed
```

### Why are the changes needed?

When there're large number of tasks in one stage, current api is hard to get taskList by status

### Does this PR introduce _any_ user-facing change?
Yes. Updated monitoring doc.

### How was this patch tested?
Added tests in `HistoryServerSuite`

Closes #28942 from warrenzhu25/SPARK-32125.

Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-16 11:31:24 +08:00
Erik Krogen cf22d947fb [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature
### What changes were proposed in this pull request?

This PR will remove references to these "blacklist" and "whitelist" terms besides the blacklisting feature as a whole, which can be handled in a separate JIRA/PR.

This touches quite a few files, but the changes are straightforward (variable/method/etc. name changes) and most quite self-contained.

### Why are the changes needed?

As per discussion on the Spark dev list, it will be beneficial to remove references to problematic language that can alienate potential community members. One such reference is "blacklist" and "whitelist". While it seems to me that there is some valid debate as to whether these terms have racist origins, the cultural connotations are inescapable in today's world.

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

In the test file `HiveQueryFileTest`, a developer has the ability to specify the system property `spark.hive.whitelist` to specify a list of Hive query files that should be tested. This system property has been renamed to `spark.hive.includelist`. The old property has been kept for compatibility, but will log a warning if used. I am open to feedback from others on whether keeping a deprecated property here is unnecessary given that this is just for developers running tests.

### How was this patch tested?

Existing tests should be suitable since no behavior changes are expected as a result of this PR.

Closes #28874 from xkrogen/xkrogen-SPARK-32036-rename-blacklists.

Authored-by: Erik Krogen <ekrogen@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-15 11:40:55 -05:00
Baohe Zhang 90b0c26b22 [SPARK-31608][CORE][WEBUI] Add a new type of KVStore to make loading UI faster
### What changes were proposed in this pull request?
Add a new class HybridStore to make the history server faster when loading event files. When rebuilding the application state from event logs, HybridStore will write data to InMemoryStore at first and use a background thread to dump data to LevelDB once the writing to InMemoryStore is completed. HybridStore is to make content serving faster by using more memory. It's only safe to enable it when the cluster is not having a heavy load.

### Why are the changes needed?
HybridStore can greatly reduce the event logs loading time, especially for large log files. In general, it has 4x - 6x UI loading speed improvement for large log files. The detailed result is shown in comments.

### Does this PR introduce any user-facing change?
This PR adds new configs `spark.history.store.hybridStore.enabled` and `spark.history.store.hybridStore.maxMemoryUsage`.

### How was this patch tested?
A test suite for HybridStore is added. I also manually tested it on 3.1.0 on mac os.

This is a follow-up for the work done by Hieu Huynh in 2019.

Closes #28412 from baohe-zhang/SPARK-31608.

Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-15 07:51:13 +09:00
HyukjinKwon 4ad9bfd53b [SPARK-32138] Drop Python 2.7, 3.4 and 3.5
### What changes were proposed in this pull request?

This PR aims to drop Python 2.7, 3.4 and 3.5.

Roughly speaking, it removes all the widely known Python 2 compatibility workarounds such as `sys.version` comparison, `__future__`. Also, it removes the Python 2 dedicated codes such as `ArrayConstructor` in Spark.

### Why are the changes needed?

 1. Unsupport EOL Python versions
 2. Reduce maintenance overhead and remove a bit of legacy codes and hacks for Python 2.
 3. PyPy2 has a critical bug that causes a flaky test, SPARK-28358 given my testing and investigation.
 4. Users can use Python type hints with Pandas UDFs without thinking about Python version
 5. Users can leverage one latest cloudpickle, https://github.com/apache/spark/pull/28950. With Python 3.8+ it can also leverage C pickle.

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

Yes, users cannot use Python 2.7, 3.4 and 3.5 in the upcoming Spark version.

### How was this patch tested?

Manually tested and also tested in Jenkins.

Closes #28957 from HyukjinKwon/SPARK-32138.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-14 11:22:44 +09:00
Holden Karau 90ac9f975b [SPARK-32004][ALL] Drop references to slave
### What changes were proposed in this pull request?

This change replaces the world slave with alternatives matching the context.

### Why are the changes needed?

There is no need to call things slave, we might as well use better clearer names.

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

Yes, the ouput JSON does change. To allow backwards compatibility this is an additive change.
The shell scripts for starting & stopping workers are renamed, and for backwards compatibility old scripts are added to call through to the new ones while printing a deprecation message to stderr.

### How was this patch tested?

Existing tests.

Closes #28864 from holdenk/SPARK-32004-drop-references-to-slave.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-13 14:05:33 -07:00
angerszhu 09789ff725 [SPARK-31226][CORE][TESTS] SizeBasedCoalesce logic will lose partition
### What changes were proposed in this pull request?

When last partition's splitFile's split size is larger then  maxSize, this partition will be lost

Origin logic error like below as 1, 2, 3, 4, 5
```scala
// 5. since index = partition.size now,  jump out of the loop , then the last partition is lost since we won't call updatePartition() again.
while (index < partitions.size) {
     //  1. we assume that when index = partitions.length -1(the last partition)
      val partition = partitions(index)
      val fileSplit =
        partition.asInstanceOf[HadoopPartition].inputSplit.value.asInstanceOf[FileSplit]
      val splitSize = fileSplit.getLength
     // 2.  if  this partition's  splitSize > maxSize
      if (currentSum + splitSize < maxSize) {
        addPartition(partition, splitSize)
        index += 1
        if (index == partitions.size) {
          updateGroups
        }
      } else {
       //  3. if currentGroup.partitions.size  >0, this situation is possiable
        if (currentGroup.partitions.size == 0) {
          addPartition(partition, splitSize)
          index += 1
        } else {
        //   4. then will just call updateGroups() here first, and index won't update in group
          updateGroups
        }
      }
    }
    groups.toArray
  }
}
```
### Why are the changes needed?
Fix bug

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

### How was this patch tested?

Manual code review.

Closes #27988 from AngersZhuuuu/SPARK-31226.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 14:48:23 -07:00
Sean Owen 3ad4863673 [SPARK-29292][SPARK-30010][CORE] Let core compile for Scala 2.13
### What changes were proposed in this pull request?

The purpose of this PR is to partly resolve SPARK-29292, and fully resolve SPARK-30010, which should allow Spark to compile vs Scala 2.13 in Spark Core and up through GraphX (not SQL, Streaming, etc).

Note that we are not trying to determine here whether this makes Spark work on 2.13 yet, just compile, as a prerequisite for assessing test outcomes. However, of course, we need to ensure that the change does not break 2.12.

The changes are, in the main, adding .toSeq and .toMap calls where mutable collections / maps are returned as Seq / Map, which are immutable by default in Scala 2.13. The theory is that it should be a no-op for Scala 2.12 (these return themselves), and required for 2.13.

There are a few non-trivial changes highlighted below.
In particular, to get Core to compile, we need to resolve SPARK-30010 which removes a deprecated SparkConf method

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

Yes, removal of the deprecated SparkConf.setAll overload, which isn't legal in Scala 2.13 anymore.

### How was this patch tested?

Existing tests. (2.13 was not _tested_; this is about getting it to compile without breaking 2.12)

Closes #28971 from srowen/SPARK-29292.1.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 14:34:02 -07:00
Kousuke Saruta ceaa3924cb [SPARK-32200][WEBUI] Redirect to the history page when accessed to /history on the HistoryServer without appliation id
### What changes were proposed in this pull request?

This PR proposes to change the HistoryServer to redirect to the history page when we access to /history without application id.

### Why are the changes needed?

In the current master, status code 400 will be returned when we access to /history.
So I wonder it's better to redirect to the history page for the better UX.

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

Yes. In the current master, if we access to /history without application id, we will see like the following page.
![history-400](https://user-images.githubusercontent.com/4736016/86649650-e9105380-c01c-11ea-93bb-78fd8d2e6f7b.png)
After this change applied, we will be redirected to the history page.

### How was this patch tested?

New test added.

Closes #29016 from sarutak/history-redirect.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 13:46:59 -07:00
HyukjinKwon b84ed4146d [SPARK-32245][INFRA] Run Spark tests in Github Actions
### What changes were proposed in this pull request?

This PR aims to run the Spark tests in Github Actions.

To briefly explain the main idea:

- Reuse `dev/run-tests.py` with SBT build
- Reuse the modules in `dev/sparktestsupport/modules.py` to test each module
- Pass the modules to test into `dev/run-tests.py` directly via `TEST_ONLY_MODULES` environment variable. For example, `pyspark-sql,core,sql,hive`.
- `dev/run-tests.py` _does not_ take the dependent modules into account but solely the specified modules to test.

Another thing to note might be `SlowHiveTest` annotation. Running the tests in Hive modules takes too much so the slow tests are extracted and it runs as a separate job. It was extracted from the actual elapsed time in Jenkins:

![Screen Shot 2020-07-09 at 7 48 13 PM](https://user-images.githubusercontent.com/6477701/87050238-f6098e80-c238-11ea-9c4a-ab505af61381.png)

So, Hive tests are separated into to jobs. One is slow test cases, and the other one is the other test cases.

_Note that_ the current GitHub Actions build virtually copies what the default PR builder on Jenkins does (without other profiles such as JDK 11, Hadoop 2, etc.). The only exception is Kinesis https://github.com/apache/spark/pull/29057/files#diff-04eb107ee163a50b61281ca08f4e4c7bR23

### Why are the changes needed?

Last week and onwards, the Jenkins machines became very unstable for many reasons:
  - Apparently, the machines became extremely slow. Almost all tests can't pass.
  - One machine (worker 4) started to have the corrupt `.m2` which fails the build.
  - Documentation build fails time to time for an unknown reason in Jenkins machine specifically. This is disabled for now at https://github.com/apache/spark/pull/29017.
  - Almost all PRs are basically blocked by this instability currently.

The advantages of using Github Actions:
  - To avoid depending on few persons who can access to the cluster.
  - To reduce the elapsed time in the build - we could split the tests (e.g., SQL, ML, CORE), and run them in parallel so the total build time will significantly reduce.
  - To control the environment more flexibly.
  - Other contributors can test and propose to fix Github Actions configurations so we can distribute this build management cost.

Note that:
- The current build in Jenkins takes _more than 7 hours_. With Github actions it takes _less than 2 hours_
- We can now control the environments especially for Python easily.
- The test and build look more stable than the Jenkins'.

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

No, dev-only change.

### How was this patch tested?

Tested at https://github.com/HyukjinKwon/spark/pull/4

Closes #29057 from HyukjinKwon/migrate-to-github-actions.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-11 13:09:06 -07:00
Pavithraramachandran d7d5bdfd79 [SPARK-32103][CORE] Support IPv6 host/port in core module
### What changes were proposed in this pull request?
In IPv6 scenario, the current logic to split hostname and port is not correct.

### Why are the changes needed?
to support IPV6 deployment scenario

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

### How was this patch tested?
UT and IPV6 spark deployment with yarn

Closes #28931 from PavithraRamachandran/ipv6_issue.

Authored-by: Pavithraramachandran <pavi.rams@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-10 13:55:20 -07:00
yi.wu 578b90cdec [SPARK-32091][CORE] Ignore timeout error when remove blocks on the lost executor
### What changes were proposed in this pull request?

This PR adds the check to see whether the executor is lost (by asking the `CoarseGrainedSchedulerBackend`) after timeout error raised in `BlockManagerMasterEndponit` due to removing blocks(e.g. RDD, broadcast, shuffle). If the executor is lost, we will ignore the error. Otherwise, throw the error.

### Why are the changes needed?

When removing blocks(e.g. RDD, broadcast, shuffle), `BlockManagerMaserEndpoint` will make RPC calls to each known `BlockManagerSlaveEndpoint` to remove the specific blocks. The PRC call sometimes could end in a timeout when the executor has been lost, but only notified the `BlockManagerMasterEndpoint` after the removing call has already happened. The timeout error could therefore fail the whole job.

In this case, we actually could just ignore the error since those blocks on the lost executor could be considered as removed already.

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

Yes. In case of users hits this issue, they will have the job executed successfully instead of throwing the exception.

### How was this patch tested?

Added unit tests.

Closes #28924 from Ngone51/ignore-timeout-error-for-inactive-executor.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-10 13:36:29 +00:00
Takuya UESHIN cfecc2030d [SPARK-32160][CORE][PYSPARK] Disallow to create SparkContext in executors
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Addes tests.

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

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:51:56 +09:00
Jungtaek Lim (HeartSaVioR) 161cf2a126 [SPARK-32024][WEBUI][FOLLOWUP] Quick fix on test failure on missing when statements
### What changes were proposed in this pull request?

This patch fixes the test failure due to the missing when statements for destination path. Note that it didn't fail on master branch, because 245aee9 got rid of size call in destination path, but still good to not depend on 245aee9.

### Why are the changes needed?

The build against branch-3.0 / branch-2.4 starts to fail after merging SPARK-32024 (#28859) and this patch will fix it.

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

No.

### How was this patch tested?

Ran modified UT against master / branch-3.0 / branch-2.4.

Closes #29046 from HeartSaVioR/QUICKFIX-SPARK-32024.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:26:38 +09:00
Warren Zhu d1d16d14bc [SPARK-31723][CORE][TEST] Reenable one test case in HistoryServerSuite
### What changes were proposed in this pull request?
Enable test("static relative links are prefixed with uiRoot (spark.ui.proxyBase)")

### Why are the changes needed?
In Jira, the failed test is another one test("ajax rendered relative links are prefixed with uiRoot (spark.ui.proxyBase)"). This test has been fixed in 6a895d0

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

### How was this patch tested?
Fix UT

Closes #28970 from warrenzhu25/31723.

Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-08 16:45:36 -07:00
Zhen Li 8e7fc04637 [SPARK-32024][WEBUI] Update ApplicationStoreInfo.size during HistoryServerDiskManager initializing
### What changes were proposed in this pull request?

Update ApplicationStoreInfo.size to real size during HistoryServerDiskManager initializing.

### Why are the changes needed?

This PR is for fixing bug [32024](https://issues.apache.org/jira/browse/SPARK-32024). We found after history server restart, below error would randomly happen: "java.lang.IllegalStateException: Disk usage tracker went negative (now = -***, delta = -***)" from `HistoryServerDiskManager`.
![Capture](https://user-images.githubusercontent.com/10524738/85034468-fda4ae80-b136-11ea-9011-f0c3e6508002.JPG)

**Cause**: Reading data from level db would trigger table file compaction, which may also trigger size of level db directory changes.  This size change may not be recorded in LevelDB (`ApplicationStoreInfo` in `listing`). When service restarts, `currentUsage` is calculated from real directory size, but `ApplicationStoreInfo` are loaded from leveldb, then `currentUsage` may be less then sum of `ApplicationStoreInfo.size`. In `makeRoom()` function, `ApplicationStoreInfo.size` is used to update usage. Then `currentUsage` becomes negative after several round of `release()` and `lease()` (`makeRoom()`).
**Reproduce**: we can reproduce this issue in dev environment by reducing config value of "spark.history.retainedApplications" and "spark.history.store.maxDiskUsage" to some small values. Here are steps: 1. start history server, load some applications and access some pages (maybe "stages" page to trigger leveldb compaction). 2. restart HS, and refresh pages.
I also added an UT to simulate this case in `HistoryServerDiskManagerSuite`.
**Benefit**: this change would help improve history server reliability.

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

No

### How was this patch tested?

Add unit test and manually tested it.

Closes #28859 from zhli1142015/update-ApplicationStoreInfo.size-during-disk-manager-initialize.

Authored-by: Zhen Li <zhli@microsoft.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-08 21:58:45 +09:00
Yuanjian Li 365961155a [SPARK-32124][CORE][FOLLOW-UP] Use the invalid value Int.MinValue to fill the map index when the event logs from the old Spark version
### What changes were proposed in this pull request?
Use the invalid value Int.MinValue to fill the map index when the event logs from the old Spark version.

### Why are the changes needed?
Follow up PR for #28941.

### Does this PR introduce _any_ user-facing change?
When we use the Spark version 3.0 history server reading the event log written by the old Spark version, we use the invalid value -2 to fill the map index.

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

Closes #28965 from xuanyuanking/follow-up.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-08 09:36:06 +09:00
sidedoorleftroad 3fe3365292 [SPARK-32172][CORE] Use createDirectory instead of mkdir
### What changes were proposed in this pull request?

Use Files.createDirectory() to create local directory instead of File.mkdir() in DiskBlockManager.
Many times, we will see such error log information like "Failed to create local dir in xxxxxx". But there is no clear information indicating why the directory creation failed.
When Files.createDirectory() fails to create a local directory, it can give specific error information for subsequent troubleshooting(also throws IOException).

### Why are the changes needed?

Throw clear error message when creating directory fails.

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

No.

### How was this patch tested?

`DiskBlockManagerSuite`

Closes #28997 from sidedoorleftroad/SPARK-32172.

Authored-by: sidedoorleftroad <sidedoorleftroad@163.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-06 09:20:31 -07:00
Dongjoon Hyun dea7bc464d [SPARK-32100][CORE][TESTS][FOLLOWUP] Reduce the required test resources
### What changes were proposed in this pull request?

This PR aims to reduce the required test resources in WorkerDecommissionExtendedSuite.

### Why are the changes needed?

When Jenkins farms is crowded, the following failure happens currently [here](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-3.2-hive-2.3/890/testReport/junit/org.apache.spark.scheduler/WorkerDecommissionExtendedSuite/Worker_decommission_and_executor_idle_timeout/)
```
java.util.concurrent.TimeoutException: Can't find 20 executors before 60000 milliseconds elapsed
	at org.apache.spark.TestUtils$.waitUntilExecutorsUp(TestUtils.scala:326)
	at org.apache.spark.scheduler.WorkerDecommissionExtendedSuite.$anonfun$new$2(WorkerDecommissionExtendedSuite.scala:45)
```

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

No.

### How was this patch tested?

Pass the Jenkins.

Closes #29001 from dongjoon-hyun/SPARK-32100-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-05 20:12:41 -07:00
Kousuke Saruta 3726aab640 [SPARK-32177][WEBUI] Remove the weird line from near the Spark logo on mouseover in the WebUI
### What changes were proposed in this pull request?

This PR changes `webui.css` to fix a style issue on moving mouse cursor on the Spark logo.

### Why are the changes needed?

In the webui, the Spark logo is on the top right side.
When we move mouse cursor on the logo, a weird underline appears near the logo.
<img width="209" alt="logo_with_line" src="https://user-images.githubusercontent.com/4736016/86542828-3c6a9f00-bf54-11ea-9b9d-cc50c12c2c9b.png">

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

Yes. After this change applied, no more weird line shown even if mouse cursor moves on the logo.
<img width="207" alt="removed-line-from-logo" src="https://user-images.githubusercontent.com/4736016/86542877-98cdbe80-bf54-11ea-8695-ee39689673ab.png">

### How was this patch tested?

By moving mouse cursor on the Spark logo and confirmed no more weird line there.

Closes #29003 from sarutak/fix-logo-underline.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-05 19:09:04 -07:00
Eren Avsarogullari f843a5bf7c [SPARK-32026][CORE][TEST] Add PrometheusServletSuite
### What changes were proposed in this pull request?

This PR aims to add `PrometheusServletSuite`.

### Why are the changes needed?

This improves the test coverage.

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

No.

### How was this patch tested?

Pass the newly added test suite.

Closes #28865 from erenavsarogullari/spark_driver_prometheus_metrics_improvement.

Authored-by: Eren Avsarogullari <erenavsarogullari@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-01 11:33:18 -07:00
TJX2014 165c948e32 [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab
### What changes were proposed in this pull request?
`formatDate` in utils.js `org/apache/spark/ui/static/utils.js` is partly refactored.

### Why are the changes needed?
In branch-2.4,task launch time is returned as html string from driver,
while in branch-3.x,this is returned in JSON Object as`Date`type  from `org.apache.spark.status.api.v1.TaskData`
Due to:
LaunchTime from jersey server in spark driver is correct, which will be converted to date string like `2020-06-28T02:57:42.605GMT` in json object, then the formatDate in utils.js treat it as date.split(".")[0].replace("T", " ").
So `2020-06-28T02:57:42.605GMT` will be converted to `2020-06-28 02:57:42`, but correct is `2020-06-28 10:57:42` in GMT+8 timezone.
![选区_071](https://user-images.githubusercontent.com/7149304/85937186-b6d36780-b933-11ea-8382-80a3891f1c2a.png)
![选区_070](https://user-images.githubusercontent.com/7149304/85937190-bcc94880-b933-11ea-8860-2083c97ea269.png)

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

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

Closes #28918 from TJX2014/master-SPARK-32068-ui-task-lauch-time-tz.

Authored-by: TJX2014 <xiaoxingstack@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-06-30 08:56:59 -05:00
yi.wu 6fcb70e0ca [SPARK-32055][CORE][SQL] Unify getReader and getReaderForRange in ShuffleManager
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #28895 from Ngone51/unify-getreader.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-29 11:37:03 +00:00
Warren Zhu 197ac3b132 [SPARK-32124][CORE][SHS] Fix taskEndReasonFromJson to handle event logs from old Spark versions
### What changes were proposed in this pull request?
Fix bug of exception when parse event log of fetch failed task end reason without `Map Index`

### Why are the changes needed?
When Spark history server read event log produced by older version of spark 2.4 (which don't have `Map Index` field), parsing of TaskEndReason will fail. This will cause TaskEnd event being ignored.

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

### How was this patch tested?
JsonProtocolSuite.test("FetchFailed Map Index backwards compatibility")

Closes #28941 from warrenzhu25/shs-task.

Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-28 21:06:45 -07:00
gengjiaan 7445c7534b [SPARK-31845][CORE][TESTS] Refactor DAGSchedulerSuite by introducing completeAndCheckAnswer and using completeNextStageWithFetchFailure
### What changes were proposed in this pull request?
**First**
`DAGSchedulerSuite` provides `completeNextStageWithFetchFailure` to make all tasks in non first stage occur `FetchFailed`.
But many test case uses complete directly as follows:
```scala
 complete(taskSets(1), Seq(
     (FetchFailed(makeBlockManagerId("hostA"),
        shuffleDep1.shuffleId, 0L, 0, 0, "ignored"), null)))
```
We need to reuse `completeNextStageWithFetchFailure`.

**Second**
`DAGSchedulerSuite` also check the results show below:
```scala
complete(taskSets(0), Seq((Success, 42)))
assert(results === Map(0 -> 42))
```
We can extract it as a generic method of `checkAnswer`.

### Why are the changes needed?
Reuse `completeNextStageWithFetchFailure`

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

### How was this patch tested?
Jenkins test

Closes #28866 from beliefer/reuse-completeNextStageWithFetchFailure.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-26 19:36:06 -07:00
Dongjoon Hyun 594cb56075 [SPARK-32100][CORE][TESTS] Add WorkerDecommissionExtendedSuite
### What changes were proposed in this pull request?

This PR aims to add `WorkerDecomissionExtendedSuite` for various worker decommission combinations.

### Why are the changes needed?

This will improve the test coverage.

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

No.

### How was this patch tested?

Pass the Jenkins.

Closes #28929 from dongjoon-hyun/SPARK-WD-TEST.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-25 16:21:14 -07:00
Max Gekk 045106e29d [SPARK-32072][CORE][TESTS] Fix table formatting with benchmark results
### What changes were proposed in this pull request?
Set column width w/ benchmark names to maximum of either
1. 40 (before this PR) or
2. The length of benchmark name or
3. Maximum length of cases names

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

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

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

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

### How was this patch tested?
By re-generating benchmark results for `MakeDateTimeBenchmark`:
```
$ SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark"
```
in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

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

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-24 04:43:53 +00:00
Zhen Li eedc6cc37d [SPARK-32028][WEBUI] fix app id link for multi attempts app in history summary page
### What changes were proposed in this pull request?

Fix app id link for multi attempts application in history summary page
If attempt id is available (yarn), app id link url will contain correct attempt id, like `/history/application_1561589317410_0002/1/jobs/`.
If attempt id is not available (standalone), app id link url will not contain fake attempt id, like `/history/app-20190404053606-0000/jobs/`.

### Why are the changes needed?

This PR is for fixing [32028](https://issues.apache.org/jira/browse/SPARK-32028). App id link use application attempt count as attempt id. this would cause link url wrong for below cases:
1. there are multi attempts, all links point to last attempt
![multi_same](https://user-images.githubusercontent.com/10524738/85098505-c45c5500-b1af-11ea-8912-fa5fd72ce064.JPG)

2. if there is one attempt, but attempt id is not 1 (before attempt maybe crash or fail to gerenerate event file). link url points to worng attempt (1) here.
![wrong_attemptJPG](https://user-images.githubusercontent.com/10524738/85098513-c9b99f80-b1af-11ea-8cbc-fd7f745c1080.JPG)

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

No

### How was this patch tested?

Tested this manually.

Closes #28867 from zhli1142015/fix-appid-link-in-history-page.

Authored-by: Zhen Li <zhli@microsoft.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-23 21:43:02 -05:00
mcheah aa4c10025a [SPARK-31798][SHUFFLE][API] Shuffle Writer API changes to return custom map output metadata
Introduces the concept of a `MapOutputMetadata` opaque object that can be returned from map output writers.

Note that this PR only proposes the API changes on the shuffle writer side. Following patches will be proposed for actually accepting the metadata on the driver and persisting it in the driver's shuffle metadata storage plugin.

### Why are the changes needed?

For a more complete design discussion on this subject as a whole, refer to [this design document](https://docs.google.com/document/d/1Aj6IyMsbS2sdIfHxLvIbHUNjHIWHTabfknIPoxOrTjk/edit#).

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

Enables additional APIs for the shuffle storage plugin tree. Usage will become more apparent as the API evolves.

### How was this patch tested?

No tests here, since this is only an API-side change that is not consumed by core Spark itself.

Closes #28616 from mccheah/return-map-output-metadata.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-21 19:47:24 -07:00
yi.wu 4badef38a5 [SPARK-32000][CORE][TESTS] Fix the flaky test for partially launched task in barrier-mode
### What changes were proposed in this pull request?

This PR changes the test to get an active executorId and set it as preferred location instead of setting a fixed preferred location.

### Why are the changes needed?

The test is flaky. After checking the [log](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124086/artifact/core/), I find the root cause is:

Two test cases from different test suites got submitted at the same time because of concurrent execution. In this particular case, the two test cases (from DistributedSuite and BarrierTaskContextSuite) both launch under local-cluster mode. The two applications are submitted at the SAME time so they have the same applications(app-20200615210132-0000). Thus, when the cluster of BarrierTaskContextSuite is launching executors, it failed to create the directory for the executor 0, because the path (/home/jenkins/workspace/work/app-app-20200615210132-0000/0) has been used by the cluster of DistributedSuite. Therefore, it has to launch executor 1 and 2 instead, that lead to non of the tasks can get preferred locality thus they got scheduled together and lead to the test failure.

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

No.

### How was this patch tested?

The test can not be reproduced locally. We can only know it's been fixed when it's no longer flaky on Jenkins.

Closes #28849 from Ngone51/fix-spark-32000.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-17 13:28:47 +00:00
Dongjoon Hyun 93bb70f3fe [SPARK-29148][CORE][FOLLOWUP] Fix warning message to show a correct executor id
### What changes were proposed in this pull request?

This aims to replace `executorIdsToBeRemoved` with `executorIdToBeRemoved`.

### Why are the changes needed?

Since a wrong variable is used currently, `ArrayBuffer()` is always displayed.
```
20/06/16 19:33:31 WARN ExecutorAllocationManager: Not removing executor ArrayBuffer() because the ResourceProfile was UNKNOWN!
```

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

No.

### How was this patch tested?

Manual.

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

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-16 21:05:29 -07:00
Zhen Li 2ec9b86628 [SPARK-31929][WEBUI] Close leveldbiterator when leveldb.close
### What changes were proposed in this pull request?

Close LevelDBIterator when LevelDB.close() is called.

### Why are the changes needed?

This pull request would prevent JNI resources leaking from Level DB instance and its' iterators. In before implementation JNI resources from LevelDBIterator are cleaned by finalize() function. This behavior is also mentioned in comments of ["LevelDBIterator.java"](https://github.com/apache/spark/blob/master/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java) by squito . But if DB instance is already closed, then iterator's close method would be ignored. LevelDB's iterator would keep level db files opened (for the case table cache is filled up), till iterator.close() is called. Then these JNI resources (file handle) would be leaked.
This JNI resource leaking issue would cause the problem described in [SPARK-31929](https://issues.apache.org/jira/browse/SPARK-31929) on Windows: in spark history server, leaked file handle for level db files would trigger "IOException" when HistoryServerDiskManager try to remove them for releasing disk space.
![IOException](https://user-images.githubusercontent.com/10524738/84134659-7c388680-aa7b-11ea-807f-04dcfa7886a0.JPG)

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

No

### How was this patch tested?

Add unit test and manually tested it.

Closes #28769 from zhli1142015/close-leveldbiterator-when-leveldb.close.

Authored-by: Zhen Li <zhli@microsoft.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-16 12:59:57 -05:00
gengjiaan 5e89fbe44e [SPARK-31824][CORE][TESTS] DAGSchedulerSuite: Improve and reuse completeShuffleMapStageSuccessfully
### What changes were proposed in this pull request?
`DAGSchedulerSuite `provides `completeShuffleMapStageSuccessfully `to make `ShuffleMapStage `successfully.
But many test case uses complete directly as follows:
`complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1))))`

We need to improve `completeShuffleMapStageSuccessfully `and reuse it.
`completeShuffleMapStageSuccessfully(0, 0, 1, Some(0), Seq("hostA"))`

### Why are the changes needed?
Improve and reuse completeShuffleMapStageSuccessfully

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

### How was this patch tested?
Jenkins test

Closes #28641 from beliefer/improve-and-reuse-method.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-06-15 14:15:54 -07:00
yi.wu 54e702c0dd [SPARK-31970][CORE] Make MDC configuration step be consistent between setLocalProperty and log4j.properties
### What changes were proposed in this pull request?

This PR proposes to use "mdc.XXX" as the consistent key for both `sc.setLocalProperty` and `log4j.properties` when setting up configurations for MDC.
### Why are the changes needed?

It's weird that we use "mdc.XXX" as key to set MDC value via `sc.setLocalProperty` while we use "XXX" as key to set MDC pattern in log4j.properties. It could also bring extra burden to the user.

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

No, as MDC feature is added in version 3.1, which hasn't been released.

### How was this patch tested?

Tested manually.

Closes #28801 from Ngone51/consistent-mdc.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-14 14:26:11 -07:00
Kousuke Saruta c2e5012a0a [SPARK-31632][CORE][WEBUI][FOLLOWUP] Enrich the exception message when application summary is unavailable
### 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.
-->
This PR enriches the exception message when application summary is not available.
#28444 covers the case when application information is not available but the case application summary is not available is not covered.

### 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.
-->
To complement #28444 .

### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
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 possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
Yes.
Before this change, we can get the following error message when we access to `/jobs` if application summary is not available.
<img width="707" alt="no-such-element-exception-error-message" src="https://user-images.githubusercontent.com/4736016/84562182-6aadf200-ad8d-11ea-8980-d63edde6fad6.png">

After this change, we can get the following error message. It's like #28444 does.
<img width="1349" alt="enriched-errorm-message" src="https://user-images.githubusercontent.com/4736016/84562189-85806680-ad8d-11ea-8346-4da2ec11df2b.png">

### 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.
-->

I checked with the following procedure.
1. Set breakpoint in the line of `kvstore.write(appSummary)` in `AppStatusListener#onStartApplicatin`. Only the thread reaching this line should be suspended.
2. Start spark-shell and wait few seconds.
3. Access to `/jobs`

Closes #28820 from sarutak/fix-no-such-element.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-14 14:17:16 +09:00
Gengliang Wang f535004e14 [SPARK-31967][UI] Downgrade to vis.js 4.21.0 to fix Jobs UI loading time regression
### What changes were proposed in this pull request?

After #28192, the job list page becomes very slow.
For example, after the following operation, the UI loading can take >40 sec.
```
(1 to 1000).foreach(_ => sc.parallelize(1 to 10).collect)
```

This is caused by a  [performance issue of `vis-timeline`](https://github.com/visjs/vis-timeline/issues/379). The serious issue affects both branch-3.0 and branch-2.4

I tried a different version 4.21.0 from https://cdnjs.com/libraries/vis
The infinite drawing issue seems also fixed if the zoom is disabled as default.

### Why are the changes needed?

Fix the serious perf issue in web UI by falling back vis-timeline-graph2d to an ealier version.

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

Yes, fix the UI perf regression

### How was this patch tested?

Manual test

Closes #28806 from gengliangwang/downgradeVis.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-06-12 17:22:41 -07:00
iRakson 9b098f1eb9 [SPARK-30119][WEBUI] Support pagination for streaming tab
### What changes were proposed in this pull request?
#28747 reverted #28439 due to some flaky test case. This PR fixes the flaky test and adds pagination support.

### Why are the changes needed?
To support pagination for streaming tab

### Does this PR introduce _any_ user-facing change?
Yes, Now streaming tab tables will be paginated.

### How was this patch tested?
Manually.

Closes #28748 from iRakson/fixstreamingpagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-12 10:27:31 -05:00
Kousuke Saruta 88a4e55fae [SPARK-31765][WEBUI][TEST-MAVEN] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

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

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-11 18:27:53 -05:00
yi.wu 91cd06bd56 [SPARK-8981][CORE][FOLLOW-UP] Clean up MDC properties after running a task
### What changes were proposed in this pull request?

This PR is a followup of #26624. This PR cleans up MDC properties if the original value is empty.
Besides, this PR adds a warning and ignore the value when the user tries to override the value of `taskName`.

### Why are the changes needed?

Before this PR, running the following jobs:

```
sc.setLocalProperty("mdc.my", "ABC")
sc.parallelize(1 to 100).count()
sc.setLocalProperty("mdc.my", null)
sc.parallelize(1 to 100).count()
```

there's still MDC value "ABC" in the log of the second count job even if we've unset the value.

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

Yes, user will 1) no longer see the MDC values after unsetting the value; 2) see a warning if he/she tries to override the value of `taskName`.

### How was this patch tested?

Tested Manaually.

Closes #28756 from Ngone51/followup-8981.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-11 14:16:12 +00:00
SaurabhChawla 82ff29be7a [SPARK-31941][CORE] Replace SparkException to NoSuchElementException for applicationInfo in AppStatusStore
### What changes were proposed in this pull request?
After SPARK-31632 SparkException is thrown from def applicationInfo
`def applicationInfo(): v1.ApplicationInfo = {
    try {
      // The ApplicationInfo may not be available when Spark is starting up.
      store.view(classOf[ApplicationInfoWrapper]).max(1).iterator().next().info
    } catch {
      case _: NoSuchElementException =>
        throw new SparkException("Failed to get the application information. " +
          "If you are starting up Spark, please wait a while until it's ready.")
    }
  }`

Where as the caller for this method def getSparkUser in Spark UI is not handling SparkException in the catch

`def getSparkUser: String = {
    try {
      Option(store.applicationInfo().attempts.head.sparkUser)
        .orElse(store.environmentInfo().systemProperties.toMap.get("user.name"))
        .getOrElse("<unknown>")
    } catch {
      case _: NoSuchElementException => "<unknown>"
    }
  }`

So On using this method (getSparkUser )we can get the application erred out.

As the part of this PR we will replace SparkException to NoSuchElementException for applicationInfo in AppStatusStore

### Why are the changes needed?
On invoking the method getSparkUser, we can get the SparkException on calling store.applicationInfo(). And this is not handled in the catch block and getSparkUser will error out in this scenario

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

### How was this patch tested?
Done the manual testing using the spark-shell and spark-submit

Closes #28768 from SaurabhChawla100/SPARK-31941.

Authored-by: SaurabhChawla <saurabhc@qubole.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-06-10 16:51:19 +09:00
yi.wu 8490eabc02 [SPARK-31486][CORE][FOLLOW-UP] Use ConfigEntry for config "spark.standalone.submit.waitAppCompletion"
### What changes were proposed in this pull request?

This PR replaces constant config with the `ConfigEntry`.

### Why are the changes needed?

For better code maintenance.

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

No.

### How was this patch tested?

Tested manually.

Closes #28775 from Ngone51/followup-SPARK-31486.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-10 16:42:38 +09:00
Takuya UESHIN 032d17933b [SPARK-31945][SQL][PYSPARK] Enable cache for the same Python function
### What changes were proposed in this pull request?

This PR proposes to make `PythonFunction` holds `Seq[Byte]` instead of `Array[Byte]` to be able to compare if the byte array has the same values for the cache manager.

### Why are the changes needed?

Currently the cache manager doesn't use the cache for `udf` if the `udf` is created again even if the functions is the same.

```py
>>> func = lambda x: x

>>> df = spark.range(1)
>>> df.select(udf(func)("id")).cache()
```
```py
>>> df.select(udf(func)("id")).explain()
== Physical Plan ==
*(2) Project [pythonUDF0#14 AS <lambda>(id)#12]
+- BatchEvalPython [<lambda>(id#0L)], [pythonUDF0#14]
 +- *(1) Range (0, 1, step=1, splits=12)
```

This is because `PythonFunction` holds `Array[Byte]`, and `equals` method of array equals only when the both array is the same instance.

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

Yes, if the user reuse the Python function for the UDF, the cache manager will detect the same function and use the cache for it.

### How was this patch tested?

I added a test case and manually.

```py
>>> df.select(udf(func)("id")).explain()
== Physical Plan ==
InMemoryTableScan [<lambda>(id)#12]
   +- InMemoryRelation [<lambda>(id)#12], StorageLevel(disk, memory, deserialized, 1 replicas)
         +- *(2) Project [pythonUDF0#5 AS <lambda>(id)#3]
            +- BatchEvalPython [<lambda>(id#0L)], [pythonUDF0#5]
               +- *(1) Range (0, 1, step=1, splits=12)
```

Closes #28774 from ueshin/issues/SPARK-31945/udf_cache.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-10 16:38:59 +09:00
yi.wu 38873d5196 [SPARK-31921][CORE] Fix the wrong warning: "App app-xxx requires more resource than any of Workers could have"
### What changes were proposed in this pull request?

This PR adds the check to see whether the allocated executors for the waiting application is empty before recognizing it as a possible hang application.

### Why are the changes needed?

It's a bugfix. The warning means there are not enough resources for the application to launch at least one executor. But we can still successfully run a job under this warning, which means it does have launched executor.

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

Yes. Before this PR, when using local cluster mode to start spark-shell, e.g. `./bin/spark-shell --master "local-cluster[2, 1, 1024]"`, the user would always see the warning:

```
20/06/06 22:21:02 WARN Utils: Your hostname, C02ZQ051LVDR resolves to a loopback address: 127.0.0.1; using 192.168.1.6 instead (on interface en0)
20/06/06 22:21:02 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/06/06 22:21:02 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly.
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly.
Spark context Web UI available at http://192.168.1.6:4040
Spark context available as 'sc' (master = local-cluster[2, 1, 1024], app id = app-20200606222107-0000).
Spark session available as 'spark'.
20/06/06 22:21:07 WARN Master: App app-20200606222107-0000 requires more resource than any of Workers could have.
20/06/06 22:21:07 WARN Master: App app-20200606222107-0000 requires more resource than any of Workers could have.
```

After this PR, the warning has gone.

### How was this patch tested?

Tested manually.

Closes #28742 from Ngone51/fix_warning.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-09 09:20:54 -07:00
Akshat Bordia 6befb2d8bd [SPARK-31486][CORE] spark.submit.waitAppCompletion flag to control spark-submit exit in Standalone Cluster Mode
### What changes were proposed in this pull request?
These changes implement an application wait mechanism which will allow spark-submit to wait until the application finishes in Standalone Spark Mode. This will delay the exit of spark-submit JVM until the job is completed. This implementation will keep monitoring the application until it is either finished, failed or killed. This will be controlled via a flag (spark.submit.waitForCompletion) which will be set to false by default.

### Why are the changes needed?
Currently, Livy API for Standalone Cluster Mode doesn't know when the job has finished. If this flag is enabled, this can be used by Livy API (/batches/{batchId}/state) to find out when the application has finished/failed. This flag is Similar to spark.yarn.submit.waitAppCompletion.

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

Yes, this PR introduces a new flag but it will be disabled by default.

### How was this patch tested?
Couldn't implement unit tests since the pollAndReportStatus method has System.exit() calls. Please provide any suggestions.
Tested spark-submit locally for the following scenarios:
1. With the flag enabled, spark-submit exits once the job is finished.
2. With the flag enabled and job failed, spark-submit exits when the job fails.
3. With the flag disabled, spark-submit exists post submitting the job (existing behavior).
4. Existing behavior is unchanged when the flag is not added explicitly.

Closes #28258 from akshatb1/master.

Lead-authored-by: Akshat Bordia <akshat.bordia31@gmail.com>
Co-authored-by: Akshat Bordia <akshat.bordia@citrix.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-09 09:29:37 -05:00
Shixiong Zhu b333ed0c4a
[SPARK-31923][CORE] Ignore internal accumulators that use unrecognized types rather than crashing
### What changes were proposed in this pull request?

Ignore internal accumulators that use unrecognized types rather than crashing so that an event log containing such accumulators can still be converted to JSON and logged.

### Why are the changes needed?

A user may use internal accumulators by adding the `internal.metrics.` prefix to the accumulator name to hide sensitive information from UI (Accumulators except internal ones will be shown in Spark UI).

However, `org.apache.spark.util.JsonProtocol.accumValueToJson` assumes an internal accumulator has only 3 possible types: `int`, `long`, and `java.util.List[(BlockId, BlockStatus)]`. When an internal accumulator uses an unexpected type, it will crash.

An event log that contains such accumulator will be dropped because it cannot be converted to JSON, and it will cause weird UI issue when rendering in Spark History Server. For example, if `SparkListenerTaskEnd` is dropped because of this issue, the user will see the task is still running even if it was finished.

It's better to make `accumValueToJson` more robust because it's up to the user to pick up the accumulator name.

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

No

### How was this patch tested?

The new unit tests.

Closes #28744 from zsxwing/fix-internal-accum.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-06-08 12:06:17 -07:00
Kousuke Saruta f7501ddd70 Revert "[SPARK-30119][WEBUI] Add Pagination Support to Streaming Page"
This PR reverts #28439 due to that PR breaks QA build.

Closes #28747 from sarutak/revert-SPARK-30119.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-06-08 10:10:52 +09:00
iRakson e9337f505b [SPARK-30119][WEBUI] Add Pagination Support to Streaming Page
### What changes were proposed in this pull request?
* Pagination Support is added to all tables of streaming page in spark web UI.
For adding pagination support, existing classes from #7399 were used.
* Earlier streaming page has two tables `Active Batches` and `Completed Batches`. Now, we will have three tables `Running Batches`, `Waiting Batches` and `Completed Batches`. If we have large number of waiting and running batches then keeping track in a single table is difficult. Also other pages have different table for different type type of data.
* Earlier empty tables were shown. Now only non-empty tables will be shown.
`Active Batches` table used to show details of waiting batches followed by running batches.

### Why are the changes needed?
Pagination will allow users to analyse the table in much better way. All spark web UI pages support pagination apart from streaming pages, so this will add consistency as well. Also it might fix the potential OOM errors that can arise.

### Does this PR introduce _any_ user-facing change?
Yes. `Active Batches` table is split into two tables `Running Batches` and `Waiting Batches`. Pagination Support is added to the all the tables. Every other functionality is unchanged.

### How was this patch tested?
Manually.

Before changes:
<img width="1667" alt="Screenshot 2020-05-03 at 7 07 14 PM" src="https://user-images.githubusercontent.com/15366835/80915680-8fb44b80-8d71-11ea-9957-c4a3769b8b67.png">

After Changes:
<img width="1669" alt="Screenshot 2020-05-03 at 6 51 22 PM" src="https://user-images.githubusercontent.com/15366835/80915694-a9ee2980-8d71-11ea-8fc5-246413a4951d.png">

Closes #28439 from iRakson/streamingPagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-06-07 13:08:50 +09:00
Kousuke Saruta 8ed93c9355 [SPARK-31886][WEBUI] Fix the wrong coloring of nodes in DAG-viz
### What changes were proposed in this pull request?

This PR fixes a wrong coloring issue in the DAG-viz.
In the Job Page and Stage Page, nodes which are associated with "barrier mode" in the DAG-viz will be colored pale green.
But, with some type of jobs, nodes which are not associated with the mode will also colored.
You can reproduce with the following operation.
```
sc.parallelize(1 to 10).barrier.mapPartitions(identity).repartition(1).collect()
```
<img width="376" alt="wrong-coloring" src="https://user-images.githubusercontent.com/4736016/83403670-1711df00-a444-11ea-9457-c683f75bc566.png">

In the screen shot above, `repartition` in `Stage 1` is not associated with barrier mode so the corresponding node should not be colored pale green.

The cause of this issue is the logic which chooses HTML elements to be colored is wrong.
The logic chooses such elements based on whether each element is associated with a style class (`clusterId` in the code).
But when an operation crosses over shuffle (like `repartition` above), a `clusterId` can be duplicated and non-barrier mode node is also associated with the same `clusterId`.

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

Newly added test case with the following command.
```
build/sbt -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite -- -z SPARK-31886"
```

Closes #28694 from sarutak/fix-wrong-barrier-color.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-06-03 01:15:36 -07:00
HyukjinKwon baafd4386c Revert "[SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0"
This reverts commit e5c3463910.
2020-06-03 14:15:30 +09:00
Kousuke Saruta 271eb26c02 [SPARK-31882][WEBUI] DAG-viz is not rendered correctly with pagination
### What changes were proposed in this pull request?

This PR fix an issue related to DAG-viz.
Because DAG-viz for a job fetches link urls for each stage from the stage table, rendering can fail with pagination.
You can reproduce this issue with the following operation.
```
sc.parallelize(1 to 10).map(value => (value ,value)).repartition(1).repartition(1).repartition(1).reduceByKey(_ + _).collect
```
And then, visit the corresponding job page.
There are 5 stages so show <5 stages in the paged table.
<img width="1440" alt="dag-rendering-issue1" src="https://user-images.githubusercontent.com/4736016/83376286-c29f3d00-a40c-11ea-891b-eb8f42afbb27.png">
<img width="1439" alt="dag-rendering-issue2" src="https://user-images.githubusercontent.com/4736016/83376288-c3d06a00-a40c-11ea-8bb2-38542e5010c1.png">

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

Newly added test case with following command.
`build/sbt -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite -- -z SPARK-31882"`

Closes #28690 from sarutak/fix-dag-rendering-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-06-02 16:45:16 -07:00
Kousuke Saruta e5c3463910 [SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

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

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-02 08:29:07 -05:00
Thomas Graves ff4a97d4b6 [SPARK-29150][CORE] Update RDD API for Stage level scheduling to be public
### What changes were proposed in this pull request?

This PR is to make the RDD api for stage level scheduling public.  All the other jiras for functionality are in so now we can make it public for people to use and tag it properly.

### Why are the changes needed?

to make it usable.

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

Yes the api is now public

### How was this patch tested?

Unit tests and manually tested.

Closes #28697 from tgravescs/SPARK-29150.

Lead-authored-by: Thomas Graves <tgraves@apache.org>
Co-authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-02 12:57:15 +09:00
Kousuke Saruta 6a895d0771 [SPARK-31804][WEBUI] Add real headless browser support for HistoryServer tests
### What changes were proposed in this pull request?

This PR adds two things.

Real headless browser support for HistoryServer tests.
A test suite using headless Chrome as one instance of those browsers.

### Why are the changes needed?

The motivation is same as #28578 .
In the current master, there is a testcase for HistoryServer which uses Ajax so we need the support for HistoryServer tests.

Also this change is necessary to upgrade HtmlUnit (See #28585)

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

No.

### How was this patch tested?

I tested with following patterns. Both Chrome and Chrome driver should be installed to test.
1. sbt / with default excluded tags (ChromeUIHistoryServerSuite is expected to be skipped and SQLQueryTestSuite is expected to succeed)
`build/sbt -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.deploy.history.ChromeUIHistoryServerSuite org.apache.spark.sql.SQLQueryTestSuite"

2. sbt / overwrite default excluded tags as empty string (Both suites are expected to succeed)
`build/sbt -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.deploy.history.ChromeUIHistoryServerSuite org.apache.spark.sql.SQLQueryTestSuite"

3. sbt / set `test.exclude.tags` to `org.apache.spark.tags.ExtendedSQLTest` (Both suites are expected to be skipped)
`build/sbt -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.deploy.history.ChromeUIHistoryServerSuite org.apache.spark.sql.SQLQueryTestSuite"

4. Maven / with default excluded tags (ChromeUIHistoryServerSuite is expected to be skipped and SQLQueryTestSuite is expected to succeed)
`build/mvn -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.ChromeUIHistoryServerSuite,org.apache.spark.sql.SQLQueryTestSuite test`

5. Maven / overwrite default excluded tags as empty string (Both suites are expected to succeed)
`build/mvn -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.ChromeUIHistoryServerSuite,org.apache.spark.sql.SQLQueryTestSuite test`

6. Maven / set `test.exclude.tags` to `org.apache.spark.tags.ExtendedSQLTest` (Both suites are expected to be skipped)
`build/mvn -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest  -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.ChromeUIHistoryServerSuite,org.apache.spark.sql.SQLQueryTestSuite test`

Closes #28622 from sarutak/headless-browser-support-for-historyserver.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-01 10:00:10 -05:00
yi.wu bc24c99af4 [SPARK-31837][CORE] Shift to the new highest locality level if there is when recomputeLocality
### What changes were proposed in this pull request?

This PR proposes to shift to the new most local locality level if there're any new more local locality levels are added during `TaskSetManager.recomputeLocality`.

### Why are the changes needed?

There's a race condition between `resourceOffers` and `submitTasks`. If `submitTasks` happens before `resourceOffers`, especially when there are no executors added to `TaskSchedulerImpl` at all, the `TaskSetManager` 's `myLocalityLevels`  will only have ANY locality level, see:

8b4862953a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (L218)

And then, `resourceOffers` is called with new executors added to `TaskSchedulerImpl`. Then, `recomputeLocality` will be called because of `executorAdded`. During `recomputeLocality`, the TaskSetManager's `myLocalityLevels` might have PROCESS_LOCAL, NODE_LOCAL, ANY(because at this time we could find alive executors from TaskSchedulerImpl). But the `TaskSetManager` will stick to the previous locality level, which is ANY, see:

8b4862953a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (L1107-L1114)

As a result, in the first round of `resourceOffers`, the new version delay scheduling won't take effect and  `TaskSetManager` can only schedule tasks at ANY level.

Please note that the problem also exists in old version delay scheduling but the impact is minor because we always reset the locality level after successfully launching a task, which is broken in the new version of dealy scheduling.

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

No.

### How was this patch tested?

Updated and added new test.

Closes #28656 from Ngone51/impr_recompute.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-01 13:06:18 +00:00
Kousuke Saruta d3eba5bc8c
[SPARK-31756][WEBUI] Add real headless browser support for UI test
### What changes were proposed in this pull request?

This PR mainly adds two things.

1. Real headless browser support for UI test
2. A test suite using headless Chrome as one instance of  those browsers.

Also, for environment where Chrome and Chrome driver is not installed, `ChromeUITest` tag is added to filter out the test suite.
By default, test suites with `ChromeUITest` is disabled.

### Why are the changes needed?

In the current master, there are two problems for UI test.
1. Lots of tests especially JavaScript related ones are done manually.
Appearance is better to be confirmed by our eyes but logic should be tested by test cases ideally.

2. Compared to the real web browsers, HtmlUnit doesn't seem to support JavaScript enough.
I added a JavaScript related test before for SPARK-31534 using HtmlUnit which is simple library based headless browser for test.
The test I added works somehow but some JavaScript related error is shown in unit-tests.log.

```
======= EXCEPTION START ========
Exception class=[net.sourceforge.htmlunit.corejs.javascript.JavaScriptException]
com.gargoylesoftware.htmlunit.ScriptException: Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)". (http://192.168.1.209:60724/static/jquery-3.4.1.min.js#2)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$HtmlUnitContextAction.run(JavaScriptEngine.java:904)
        at net.sourceforge.htmlunit.corejs.javascript.Context.call(Context.java:628)
        at net.sourceforge.htmlunit.corejs.javascript.ContextFactory.call(ContextFactory.java:515)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine.callFunction(JavaScriptEngine.java:835)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine.callFunction(JavaScriptEngine.java:807)
        at com.gargoylesoftware.htmlunit.InteractivePage.executeJavaScriptFunctionIfPossible(InteractivePage.java:216)
        at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptFunctionJob.runJavaScript(JavaScriptFunctionJob.java:52)
        at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptExecutionJob.run(JavaScriptExecutionJob.java:102)
        at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptJobManagerImpl.runSingleJob(JavaScriptJobManagerImpl.java:426)
        at com.gargoylesoftware.htmlunit.javascript.background.DefaultJavaScriptExecutor.run(DefaultJavaScriptExecutor.java:157)
        at java.lang.Thread.run(Thread.java:748)
Caused by: net.sourceforge.htmlunit.corejs.javascript.JavaScriptException: Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)". (http://192.168.1.209:60724/static/jquery-3.4.1.min.js#2)
        at net.sourceforge.htmlunit.corejs.javascript.Interpreter.interpretLoop(Interpreter.java:1009)
        at net.sourceforge.htmlunit.corejs.javascript.Interpreter.interpret(Interpreter.java:800)
        at net.sourceforge.htmlunit.corejs.javascript.InterpretedFunction.call(InterpretedFunction.java:105)
        at net.sourceforge.htmlunit.corejs.javascript.ContextFactory.doTopCall(ContextFactory.java:413)
        at com.gargoylesoftware.htmlunit.javascript.HtmlUnitContextFactory.doTopCall(HtmlUnitContextFactory.java:252)
        at net.sourceforge.htmlunit.corejs.javascript.ScriptRuntime.doTopCall(ScriptRuntime.java:3264)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$4.doRun(JavaScriptEngine.java:828)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$HtmlUnitContextAction.run(JavaScriptEngine.java:889)
        ... 10 more
JavaScriptException value = Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)".
== CALLING JAVASCRIPT ==
  function () {
      throw e;
  }
======= EXCEPTION END ========
```
I tried to upgrade HtmlUnit to 2.40.0 but what is worse, the test become not working even though it works on real browsers like Chrome, Safari and Firefox without error.
```
[info] UISeleniumSuite:
[info] - SPARK-31534: text for tooltip should be escaped *** FAILED *** (17 seconds, 745 milliseconds)
[info]   The code passed to eventually never returned normally. Attempted 2 times over 12.910785232 seconds. Last failure message: com.gargoylesoftware.htmlunit.ScriptException: ReferenceError: Assignment to undefined "regeneratorRuntime" in strict mode (http://192.168.1.209:62132/static/vis-timeline-graph2d.min.js#52(Function)#1)
```
To resolve those problems, it's better to support headless browser for UI test.

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

No.

### How was this patch tested?

I tested with following patterns. Both Chrome and Chrome driver should be installed to test.

1. sbt / with default excluded tags (ChromeUISeleniumSuite is expected to be skipped and SQLQueryTestSuite is expected to succeed)
`build/sbt -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite org.apache.spark.sql.SQLQueryTestSuite"

2. sbt / overwrite default excluded tags as empty string (Both suites are expected to succeed)
`build/sbt -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite org.apache.spark.sql.SQLQueryTestSuite"

3. sbt / set `test.exclude.tags` to `org.apache.spark.tags.ExtendedSQLTest` (Both suites are expected to be skipped)
`build/sbt -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite org.apache.spark.sql.SQLQueryTestSuite"

4. Maven / with default excluded tags (ChromeUISeleniumSuite is expected to be skipped and SQLQueryTestSuite is expected to succeed)
`build/mvn -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite,org.apache.spark.sql.SQLQueryTestSuite test`

5. Maven / overwrite default excluded tags as empty string (Both suites are expected to succeed)
`build/mvn -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite,org.apache.spark.sql.SQLQueryTestSuite test`

6. Maven / set `test.exclude.tags` to `org.apache.spark.tags.ExtendedSQLTest` (Both suites are expected to be skipped)
`build/mvn -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest  -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite,org.apache.spark.sql.SQLQueryTestSuite test`

Closes #28627 from sarutak/real-headless-browser-support-take2.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-29 10:41:29 -07:00
Xingbo Jiang efe7fd2b6b [SPARK-31730][CORE][TEST] Fix flaky tests in BarrierTaskContextSuite
### What changes were proposed in this pull request?

To wait until all the executors have started before submitting any job. This could avoid the flakiness caused by waiting for executors coming up.

### How was this patch tested?

Existing tests.

Closes #28584 from jiangxb1987/barrierTest.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-05-27 16:37:02 -07:00
Kousuke Saruta d19b173b47 [SPARK-31764][CORE] JsonProtocol doesn't write RDDInfo#isBarrier
### What changes were proposed in this pull request?

This PR changes JsonProtocol to write RDDInfos#isBarrier.

### Why are the changes needed?

JsonProtocol reads RDDInfos#isBarrier but not write it so it's a bug.

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

No.

### How was this patch tested?

I added a testcase.

Closes #28583 from sarutak/SPARK-31764.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-05-27 14:36:12 -07:00
iRakson 765105b6f1 [SPARK-31638][WEBUI] Clean Pagination code for all webUI pages
### What changes were proposed in this pull request?

Pagination code across pages needs to be cleaned.
I have tried to clear out these things :
* Unused methods
* Unused method arguments
* remove redundant `if` expressions
* fix indentation

### Why are the changes needed?
This fix will make code more readable and remove unnecessary methods and variables.

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

### How was this patch tested?
Manually

Closes #28448 from iRakson/refactorPagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-27 08:59:08 -05:00
Kousuke Saruta 8441e936fc Revert "[SPARK-31756][WEBUI] Add real headless browser support for UI test
This reverts commit d95570864a.

Closes #28624 from sarutak/revert-real-headless-browser-support.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-05-24 09:13:38 +09:00
Gengliang Wang 9fdc2a0801 [SPARK-31793][SQL] Reduce the memory usage in file scan location metadata
### What changes were proposed in this pull request?

Currently, the data source scan node stores all the paths in its metadata. The metadata is kept when a SparkPlan is converted into SparkPlanInfo. SparkPlanInfo can be used to construct the Spark plan graph in UI.

However, the paths can be very large (e.g. it can be many partitions after partition pruning), while UI pages only require up to 100 bytes for the location metadata. We can reduce the paths stored in metadata to reduce memory usage.

### Why are the changes needed?

Reduce unnecessary memory cost.
In the heap dump of a driver, the SparkPlanInfo instances are quite large and it should be avoided:
![image](https://user-images.githubusercontent.com/1097932/82642318-8f65de00-9bc2-11ea-9c9c-f05c2b0e1c49.png)

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

No

### How was this patch tested?

Unit tests

Closes #28610 from gengliangwang/improveLocationMetadata.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-23 15:00:28 -07:00
Holden Karau 721cba5402 [SPARK-31791][CORE][TEST] Improve cache block migration test reliability
### What changes were proposed in this pull request?

Increase the timeout and register the listener earlier to avoid any race condition of the job starting before the listener is registered.

### Why are the changes needed?

The test is currently semi-flaky.

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

No

### How was this patch tested?
I'm currently running the following bash script on my dev machine to verify the flakiness decreases. It has gotten to 356 iterations without any test failures so I believe issue is fixed.

```
set -ex
./build/sbt clean compile package
((failures=0))
for (( i=0;i<1000;++i )); do
  echo "Run $i"
  ((failed=0))
  ./build/sbt "core/testOnly org.apache.spark.scheduler.WorkerDecommissionSuite" || ((failed=1))
  echo "Resulted in $failed"
  ((failures=failures+failed))
  echo "Current status is failures: $failures out of $i runs"
done
```

Closes #28614 from holdenk/SPARK-31791-improve-cache-block-migration-test-reliability.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-05-22 18:19:41 -07:00
Kousuke Saruta d95570864a [SPARK-31756][WEBUI] Add real headless browser support for UI test
### What changes were proposed in this pull request?

This PR mainly adds two things.

1. Real headless browser support for UI test
2. A test suite using headless Chrome as one instance of  those browsers.

Also, for environment where Chrome and Chrome driver is not installed, `ChromeUITest` tag is added to filter out the test suite.

### Why are the changes needed?

In the current master, there are two problems for UI test.
1. Lots of tests especially JavaScript related ones are done manually.
Appearance is better to be confirmed by our eyes but logic should be tested by test cases ideally.

2. Compared to the real web browsers, HtmlUnit doesn't seem to support JavaScript enough.
I added a JavaScript related test before for SPARK-31534 using HtmlUnit which is simple library based headless browser for test.
The test I added works somehow but some JavaScript related error is shown in unit-tests.log.

```
======= EXCEPTION START ========
Exception class=[net.sourceforge.htmlunit.corejs.javascript.JavaScriptException]
com.gargoylesoftware.htmlunit.ScriptException: Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)". (http://192.168.1.209:60724/static/jquery-3.4.1.min.js#2)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$HtmlUnitContextAction.run(JavaScriptEngine.java:904)
        at net.sourceforge.htmlunit.corejs.javascript.Context.call(Context.java:628)
        at net.sourceforge.htmlunit.corejs.javascript.ContextFactory.call(ContextFactory.java:515)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine.callFunction(JavaScriptEngine.java:835)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine.callFunction(JavaScriptEngine.java:807)
        at com.gargoylesoftware.htmlunit.InteractivePage.executeJavaScriptFunctionIfPossible(InteractivePage.java:216)
        at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptFunctionJob.runJavaScript(JavaScriptFunctionJob.java:52)
        at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptExecutionJob.run(JavaScriptExecutionJob.java:102)
        at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptJobManagerImpl.runSingleJob(JavaScriptJobManagerImpl.java:426)
        at com.gargoylesoftware.htmlunit.javascript.background.DefaultJavaScriptExecutor.run(DefaultJavaScriptExecutor.java:157)
        at java.lang.Thread.run(Thread.java:748)
Caused by: net.sourceforge.htmlunit.corejs.javascript.JavaScriptException: Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)". (http://192.168.1.209:60724/static/jquery-3.4.1.min.js#2)
        at net.sourceforge.htmlunit.corejs.javascript.Interpreter.interpretLoop(Interpreter.java:1009)
        at net.sourceforge.htmlunit.corejs.javascript.Interpreter.interpret(Interpreter.java:800)
        at net.sourceforge.htmlunit.corejs.javascript.InterpretedFunction.call(InterpretedFunction.java:105)
        at net.sourceforge.htmlunit.corejs.javascript.ContextFactory.doTopCall(ContextFactory.java:413)
        at com.gargoylesoftware.htmlunit.javascript.HtmlUnitContextFactory.doTopCall(HtmlUnitContextFactory.java:252)
        at net.sourceforge.htmlunit.corejs.javascript.ScriptRuntime.doTopCall(ScriptRuntime.java:3264)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$4.doRun(JavaScriptEngine.java:828)
        at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$HtmlUnitContextAction.run(JavaScriptEngine.java:889)
        ... 10 more
JavaScriptException value = Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)".
== CALLING JAVASCRIPT ==
  function () {
      throw e;
  }
======= EXCEPTION END ========
```
I tried to upgrade HtmlUnit to 2.40.0 but what is worse, the test become not working even though it works on real browsers like Chrome, Safari and Firefox without error.
```
[info] UISeleniumSuite:
[info] - SPARK-31534: text for tooltip should be escaped *** FAILED *** (17 seconds, 745 milliseconds)
[info]   The code passed to eventually never returned normally. Attempted 2 times over 12.910785232 seconds. Last failure message: com.gargoylesoftware.htmlunit.ScriptException: ReferenceError: Assignment to undefined "regeneratorRuntime" in strict mode (http://192.168.1.209:62132/static/vis-timeline-graph2d.min.js#52(Function)#1)
```
To resolve those problems, it's better to support headless browser for UI test.

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

No.

### How was this patch tested?

I tested with following patterns. Both Chrome and Chrome driver should be installed to test.

1. sbt / with chromedriver / include tag (expect to succeed)
`build/sbt -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite"`
2. sbt / with chromedriver / exclude tag (expect to be ignored)
`build/sbt -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite -l org.apache.spark.tags.ChromeUITest"`
3. sbt / without chromedriver / include tag (expect to be failed)
`build/sbt  "testOnly org.apache.spark.ui.ChromeUISeleniumSuite"`
4. sbt / without chromedriver / exclude tag (expect to be skipped)
`build/sbt  -Dtest.exclude.tags=org.apache.spark.tags.ChromeUITest "testOnly org.apache.spark.ui.ChromeUISeleniumSuite"`
5. Maven / wth chromedriver / include tag (expect to succeed)
`build/mvn -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite test`
6. Maven / with chromedriver / exclude tag (expect to be skipped)
`build/mvn -Dtest.exclude.tags="org.apache.spark.tags.ChromeUITest" -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite test`
7. Maven / without chromedriver / include tag (expect to be failed)
`build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite test`
8. Maven / without chromedriver / exclude tag (expect to be skipped)
`build/mvn -Dtest.exclude.tags=org.apache.spark.tags.ChromeUITest  -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite test`

Closes #28578 from sarutak/real-headless-browser-support.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-22 08:24:31 -05:00
Xingbo Jiang 245aee9fc8 [SPARK-31757][CORE] Improve HistoryServerDiskManager.updateAccessTime()
### What changes were proposed in this pull request?

The function `HistoryServerDiskManager`.`updateAccessTime()` would recompute the application store directory size every time it's triggered, this effort could be avoided because we already computed the new size outside the function call.

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

No

### How was this patch tested?

Existing test cases.

Closes #28579 from jiangxb1987/updateInfo.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-05-21 23:36:55 -07:00
yi.wu 83d0967dcc [SPARK-31784][CORE][TEST] Fix test BarrierTaskContextSuite."share messages with allGather() call"
### What changes were proposed in this pull request?

Change from `messages.toList.iterator` to `Iterator.single(messages.toList)`.

### Why are the changes needed?

In this test, the expected result of `rdd2.collect().head` should actually be `List("0", "1", "2", "3")` but is `"0"` now.

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

No.

### How was this patch tested?

Updated test.

Thanks WeichenXu123 reported this problem.

Closes #28596 from Ngone51/fix_allgather_test.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-05-21 23:34:11 -07:00
Gengliang Wang db5e5fce68 Revert "[SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0"
This reverts commit 92877c4ef2.

Closes #28602 from gengliangwang/revertSPARK-31765.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-21 16:00:58 -07:00
Kousuke Saruta 92877c4ef2 [SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

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

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-21 11:43:25 -07:00
Thomas Graves b64688ebba [SPARK-29303][WEB UI] Add UI support for stage level scheduling
### What changes were proposed in this pull request?

This adds UI updates to support stage level scheduling and ResourceProfiles. 3 main things have been added. ResourceProfile id added to the Stage page, the Executors page now has an optional selectable column to show the ResourceProfile Id of each executor, and the Environment page now has a section with the ResourceProfile ids.  Along with this the rest api for environment page was updated to include the Resource profile information.

I debating on splitting the resource profile information into its own page but I wasn't sure it called for a completely separate page. Open to peoples thoughts on this.

Screen shots:
![Screen Shot 2020-04-01 at 3 07 46 PM](https://user-images.githubusercontent.com/4563792/78185169-469a7000-7430-11ea-8b0c-d9ede2d41df8.png)
![Screen Shot 2020-04-01 at 3 08 14 PM](https://user-images.githubusercontent.com/4563792/78185175-48fcca00-7430-11ea-8d1d-6b9333700f32.png)
![Screen Shot 2020-04-01 at 3 09 03 PM](https://user-images.githubusercontent.com/4563792/78185176-4a2df700-7430-11ea-92d9-73c382bb0f32.png)
![Screen Shot 2020-04-01 at 11 05 48 AM](https://user-images.githubusercontent.com/4563792/78185186-4dc17e00-7430-11ea-8962-f749dd47ea60.png)

### Why are the changes needed?

For user to be able to know what resource profile was used with which stage and executors. The resource profile information is also available so user debugging can see exactly what resources were requested with that profile.

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

Yes, UI updates.

### How was this patch tested?

Unit tests and tested on yarn both active applications and with the history server.

Closes #28094 from tgravescs/SPARK-29303-pr.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-05-21 13:11:35 -05:00
iRakson f1495c5bc0 [SPARK-31688][WEBUI] Refactor Pagination framework
### What changes were proposed in this pull request?
Currently while implementing pagination using the existing pagination framework, a lot of code is being copied as pointed out [here](https://github.com/apache/spark/pull/28485#pullrequestreview-408881656).

I introduced some changes in `PagedTable` which is the main trait for implementing the pagination.
* Added function for getting table parameters.
* Added a function for table header row. This will help in maintaining consistency across the tables. All the header rows across tables will be consistent now.

### Why are the changes needed?

* A lot of code is copied every time pagination is implemented for any table.
* Code readability is not great as lot of HTML is embedded.
* Paginating other tables will be a lot easier now.

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

### How was this patch tested?
Manually. This is mainly refactoring work, no new functionality introduced. Existing test cases should pass.

Closes #28512 from iRakson/refactorPaginationFramework.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-21 13:00:00 -05:00
yi.wu 26bc690722 [SPARK-30689][CORE][FOLLOW-UP] Add @since annotation for ResourceDiscoveryScriptPlugin/ResourceInformation
### What changes were proposed in this pull request?

Added `since 3.0.0` for `ResourceDiscoveryScriptPlugin` and `ResourceInformation`.

### Why are the changes needed?

It's required for exposed APIs(https://github.com/apache/spark/pull/27689#discussion_r426488851).

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

Yes, they can easily know when does Spark introduces the API.

### How was this patch tested?

Pass Jenkins.

Closes #28591 from Ngone51/followup-30689.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-21 03:11:20 +09:00
Izek Greenfield eaf7a2a4ed [SPARK-8981][CORE][TEST-HADOOP3.2][TEST-JAVA11] Add MDC support in Executor
### What changes were proposed in this pull request?
Added MDC support in all thread pools.
ThreaddUtils create new pools that pass over MDC.

### Why are the changes needed?
In many cases, it is very hard to understand from which actions the logs in the executor come from.
when you are doing multi-thread work in the driver and send actions in parallel.

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

### How was this patch tested?
No test added because no new functionality added it is thread pull change and all current tests pass.

Closes #26624 from igreenfield/master.

Authored-by: Izek Greenfield <igreenfield@axiomsl.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-20 07:41:00 +00:00
Kent Yao 1f29f1ba58 [SPARK-31684][SQL] Overwrite partition failed with 'WRONG FS' when the target partition is not belong to the filesystem as same as the table
### What changes were proposed in this pull request?

With SPARK-18107, we will disable the underlying replace(overwrite) and instead do delete in spark side and only do copy in hive side to bypass the performance issue - [HIVE-11940](https://issues.apache.org/jira/browse/HIVE-11940)

Conditionally, if the table location and partition location do not belong to the same `FileSystem`, We should not disable hive overwrite. Otherwise, hive will use the `FileSystem` instance belong to the table location to copy files, which will fail in `FileSystem#checkPath`
https://github.com/apache/hive/blob/rel/release-2.3.7/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java#L1657

In this PR, for Hive 2.0.0 and onwards, as [HIVE-11940](https://issues.apache.org/jira/browse/HIVE-11940) has been fixed, and there is no performance issue anymore. We should leave the overwrite logic to hive to avoid failure in `FileSystem#checkPath`

**NOTE THAT**  For Hive 2.2.0 and earlier, if the table and partition locations do not belong together, we will still get the same error thrown by hive encryption check due to  [HIVE-14380]( https://issues.apache.org/jira/browse/HIVE-14380) which need to fix in another ticket SPARK-31675.

### Why are the changes needed?

bugfix. a logic table can be decoupled with the storage layer and may contain data from remote storage systems.

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

no

### How was this patch tested?

Currently verified manually. add benchmark tests

```sql
-INSERT INTO DYNAMIC                                7742           7918         248          0.0      756044.0       1.0X
-INSERT INTO HYBRID                                 1289           1307          26          0.0      125866.3       6.0X
-INSERT INTO STATIC                                  371            393          38          0.0       36219.4      20.9X
-INSERT OVERWRITE DYNAMIC                           8456           8554         138          0.0      825790.3       0.9X
-INSERT OVERWRITE HYBRID                            1303           1311          12          0.0      127198.4       5.9X
-INSERT OVERWRITE STATIC                             434            447          13          0.0       42373.8      17.8X
+INSERT INTO DYNAMIC                                7382           7456         105          0.0      720904.8       1.0X
+INSERT INTO HYBRID                                 1128           1129           1          0.0      110169.4       6.5X
+INSERT INTO STATIC                                  349            370          39          0.0       34095.4      21.1X
+INSERT OVERWRITE DYNAMIC                           8149           8362         301          0.0      795821.8       0.9X
+INSERT OVERWRITE HYBRID                            1317           1318           2          0.0      128616.7       5.6X
+INSERT OVERWRITE STATIC                             387            408          37          0.0       37804.1      19.1X
```

+ for master
- for this PR

both using hive 2.3.7

Closes #28511 from yaooqinn/SPARK-31684.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-19 14:08:51 +00:00
yi.wu 653ca19b1f [SPARK-31651][CORE] Improve handling the case where different barrier sync types in a single sync
### What changes were proposed in this pull request?

This PR improves handling the case where different barrier sync types in a single sync:

- use `clear` instead of `cleanupBarrierStage `

- make sure all requesters are failed because of "different barrier sync types"

### Why are the changes needed?

Currently, we use `cleanupBarrierStage` to clean up a barrier stage when we detecting the case of "different barrier sync types". But this leads to a problem that we could create new a `ContextBarrierState` for the same stage again if there're on-way requests from tasks. As a result, those task will fail because of killing instead of "different barrier sync types".

Besides, we don't handle the current request which is being handling properly as it will fail due to epoch mismatch instead of "different barrier sync types".

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

No.

### How was this patch tested?

Updated a existed test.

Closes #28462 from Ngone51/impr_barrier_req.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-05-18 23:54:41 -07:00
Prakhar Jain c560428fe0 [SPARK-20732][CORE] Decommission cache blocks to other executors when an executor is decommissioned
### What changes were proposed in this pull request?
After changes in SPARK-20628, CoarseGrainedSchedulerBackend can decommission an executor and stop assigning new tasks on it. We should also decommission the corresponding blockmanagers in the same way. i.e. Move the cached RDD blocks from those executors to other active executors.

### Why are the changes needed?
We need to gracefully decommission the block managers so that the underlying RDD cache blocks are not lost in case the executors are taken away forcefully after some timeout (because of spotloss/pre-emptible VM etc). Its good to save as much cache data as possible.

Also In future once the decommissioning signal comes from Cluster Manager (say YARN/Mesos etc), dynamic allocation + this change gives us opportunity to downscale the executors faster by making the executors free of cache data.

Note that this is a best effort approach. We try to move cache blocks from decommissioning executors to active executors. If the active executors don't have free resources available on them for caching, then the decommissioning executors will keep the cache block which it was not able to move and it will still be able to serve them.

Current overall Flow:

1. CoarseGrainedSchedulerBackend receives a signal to decommissionExecutor. On receiving the signal, it do 2 things - Stop assigning new tasks (SPARK-20628), Send another message to BlockManagerMasterEndpoint (via BlockManagerMaster) to decommission the corresponding BlockManager.

2. BlockManagerMasterEndpoint receives "DecommissionBlockManagers" message. On receiving this, it moves the corresponding block managers to "decommissioning" state. All decommissioning BMs are excluded from the getPeers RPC call which is used for replication. All these decommissioning BMs are also sent message from BlockManagerMasterEndpoint to start decommissioning process on themselves.

3. BlockManager on worker (say BM-x) receives the "DecommissionBlockManager" message. Now it will start BlockManagerDecommissionManager thread to offload all the RDD cached blocks. This thread can make multiple reattempts to decommission the existing cache blocks (multiple reattempts might be needed as there might not be sufficient space in other active BMs initially).

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

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

Closes #28370 from prakharjain09/SPARK-20732-rddcache-1.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-05-18 11:37:53 -07:00
Kris Mok dc01b7556f [SPARK-31399][CORE][TEST-HADOOP3.2][TEST-JAVA11] Support indylambda Scala closure in ClosureCleaner
### What changes were proposed in this pull request?

This PR proposes to enhance Spark's `ClosureCleaner` to support "indylambda" style of Scala closures to the same level as the existing implementation for the old (inner class) style ones. The goal is to reach feature parity with the support of the old style Scala closures, with as close to bug-for-bug compatibility as possible.

Specifically, this PR addresses one lacking support for indylambda closures vs the inner class closures:
- When a closure is declared in a Scala REPL and captures the enclosing REPL line object, such closure should be cleanable (unreferenced fields on the enclosing REPL line object should be cleaned)

This PR maintains the same limitations in the new indylambda closure support as the old inner class closures, in particular the following two:
- Cleaning is only available for one level of REPL line object. If a closure captures state from a REPL line object further out from the immediate enclosing one, it won't be subject to cleaning. See example below.
- "Sibling" closures are not handled yet. A "sibling" closure is defined here as a closure that is directly or indirectly referenced by the starting closure, but isn't lexically enclosing. e.g.
  ```scala
  {
    val siblingClosure = (x: Int) => x + this.fieldA   // captures `this`, references `fieldA` on `this`.
    val startingClosure = (y: Int) => y + this.fieldB + siblingClosure(y)  // captures `this` and `siblingClosure`, references `fieldB` on `this`.
  }
  ```

The changes are intended to be minimal, with further code cleanups planned in separate PRs.

Jargons:
- old, inner class style Scala closures, aka `delambdafy:inline`: default in Scala 2.11 and before
- new, "indylambda" style Scala closures, aka `delambdafy:method`: default in Scala 2.12 and later

### Why are the changes needed?

There had been previous effortsto extend Spark's `ClosureCleaner` to support "indylambda" Scala closures, which is necessary for proper Scala 2.12 support. Most notably the work done for [SPARK-14540](https://issues.apache.org/jira/browse/SPARK-14540).

But the previous efforts had missed one import scenario: a Scala closure declared in a Scala REPL, and it captures the enclosing `this` -- a REPL line object. e.g. in a Spark Shell:
```scala
:pa
class NotSerializableClass(val x: Int)
val ns = new NotSerializableClass(42)
val topLevelValue = "someValue"
val func = (j: Int) => {
  (1 to j).flatMap { x =>
    (1 to x).map { y => y + topLevelValue }
  }
}
<Ctrl+D>
sc.parallelize(0 to 2).map(func).collect
```
In this example, `func` refers to a Scala closure that captures the enclosing `this` because it needs to access `topLevelValue`, which is in turn implemented as a field on the enclosing REPL line object.

The existing `ClosureCleaner` in Spark supports cleaning this case in Scala 2.11-, and this PR brings feature parity to Scala 2.12+.

Note that the existing cleaning logic only supported one level of REPL line object nesting. This PR does not go beyond that. When a closure references state declared a few commands earlier, the cleaning will fail in both Scala 2.11 and Scala 2.12. e.g.
```scala
scala> :pa
// Entering paste mode (ctrl-D to finish)

class NotSerializableClass1(val x: Int)
case class Foo(id: String)
val ns = new NotSerializableClass1(42)
val topLevelValue = "someValue"

// Exiting paste mode, now interpreting.

defined class NotSerializableClass1
defined class Foo
ns: NotSerializableClass1 = NotSerializableClass1615b1baf
topLevelValue: String = someValue

scala> :pa
// Entering paste mode (ctrl-D to finish)

val closure2 = (j: Int) => {
  (1 to j).flatMap { x =>
    (1 to x).map { y => y + topLevelValue } // 2 levels
  }
}

// Exiting paste mode, now interpreting.

closure2: Int => scala.collection.immutable.IndexedSeq[String] = <function1>

scala> sc.parallelize(0 to 2).map(closure2).collect
org.apache.spark.SparkException: Task not serializable
...
```
in the Scala 2.11 / Spark 2.4.x case:
```
Caused by: java.io.NotSerializableException: NotSerializableClass1
Serialization stack:
	- object not serializable (class: NotSerializableClass1, value: NotSerializableClass1615b1baf)
	- field (class: $iw, name: ns, type: class NotSerializableClass1)
	- object (class $iw, $iw64df3f4b)
	- field (class: $iw, name: $iw, type: class $iw)
	- object (class $iw, $iw66e6e5e9)
	- field (class: $line14.$read, name: $iw, type: class $iw)
	- object (class $line14.$read, $line14.$readc310aa3)
	- field (class: $iw, name: $line14$read, type: class $line14.$read)
	- object (class $iw, $iw79224636)
	- field (class: $iw, name: $outer, type: class $iw)
	- object (class $iw, $iw636d4cdc)
	- field (class: $anonfun$1, name: $outer, type: class $iw)
	- object (class $anonfun$1, <function1>)
```
in the Scala 2.12 / Spark master case after this PR:
```
Caused by: java.io.NotSerializableException: NotSerializableClass1
Serialization stack:
	- object not serializable (class: NotSerializableClass1, value: NotSerializableClass16f3b4c9a)
	- field (class: $iw, name: ns, type: class NotSerializableClass1)
	- object (class $iw, $iw2945a3c1)
	- field (class: $iw, name: $iw, type: class $iw)
	- object (class $iw, $iw152705d0)
	- field (class: $line14.$read, name: $iw, type: class $iw)
	- object (class $line14.$read, $line14.$read7cf311eb)
	- field (class: $iw, name: $line14$read, type: class $line14.$read)
	- object (class $iw, $iwd980dac)
	- field (class: $iw, name: $outer, type: class $iw)
	- object (class $iw, $iw557d9532)
	- element of array (index: 0)
	- array (class [Ljava.lang.Object;, size 1)
	- field (class: java.lang.invoke.SerializedLambda, name: capturedArgs, type: class [Ljava.lang.Object;)
	- object (class java.lang.invoke.SerializedLambda, SerializedLambda[capturingClass=class $iw, functionalInterfaceMethod=scala/Function1.apply:(Ljava/lang/Object;)Ljava/lang/Object;, implementation=invokeStatic $anonfun$closure2$1$adapted:(L$iw;Ljava/lang/Object;)Lscala/collection/immutable/IndexedSeq;, instantiatedMethodType=(Ljava/lang/Object;)Lscala/collection/immutable/IndexedSeq;, numCaptured=1])
	- writeReplace data (class: java.lang.invoke.SerializedLambda)
	- object (class $Lambda$2103/815179920, $Lambda$2103/815179920569b57c4)
```

For more background of the new and old ways Scala lowers closures to Java bytecode, please see [A note on how NSC (New Scala Compiler) lowers lambdas](https://gist.github.com/rednaxelafx/e9ecd09bbd1c448dbddad4f4edf25d48#file-notes-md).

For more background on how Spark's `ClosureCleaner` works and what's needed to make it support "indylambda" Scala closures, please refer to [A Note on Apache Spark's ClosureCleaner](https://gist.github.com/rednaxelafx/e9ecd09bbd1c448dbddad4f4edf25d48#file-spark_closurecleaner_notes-md).

#### tl;dr

The `ClosureCleaner` works like a mark-sweep algorithm on fields:
- Finding (a chain of) outer objects referenced by the starting closure;
- Scanning the starting closure and its inner closures and marking the fields on the outer objects accessed;
- Cloning the outer objects, nulling out fields that are not accessed by any closure of concern.

##### Outer Objects

For the old, inner class style Scala closures, the "outer objects" is defined as the lexically enclosing closures of the starting closure, plus an optional enclosing REPL line object if these closures are defined in a Scala REPL. All of them are on a singly-linked `$outer` chain.

For the new, "indylambda" style Scala closures, the capturing implementation changed, so closures no longer refer to their enclosing closures via an `$outer` chain. However, a closure can still capture its enclosing REPL line object, much like the old style closures. The name of the field that captures this reference would be `arg$1` (instead of `$outer`).

So what's missing in the `ClosureCleaner` for the "indylambda" support is find and potentially clone+clean the captured enclosing `this` REPL line object. That's what this PR implements.

##### Inner Closures

The old, inner class style of Scala closures are compiled into separate inner classes, one per lambda body. So in order to discover the implementation (bytecode) of the inner closures, one has to jump over multiple classes. The name of such a class would contain the marker substring `$anonfun$`.

The new, "indylambda" style Scala closures are compiled into **static methods** in the class where the lambdas were declared. So for lexically nested closures, their lambda bodies would all be compiled into static methods **in the same class**. This makes it much easier to discover the implementation (bytecode) of the nested lambda bodies. The name of such a static method would contain the marker substring `$anonfun$`.

Discovery of inner closures involves scanning bytecode for certain patterns that represent the creation of a closure object for the inner closure.
- For inner class style: the closure object creation site is like `new <InnerClassForTheClosure>(captured args)`
- For "indylambda" style: the closure object creation site would be compiled into an `invokedynamic` instruction, with its "bootstrap method" pointing to the same one used by Java 8 for its serializable lambdas, and with the bootstrap method arguments pointing to the implementation method.

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

Yes. Before this PR, Spark 2.4 / 3.0 / master on Scala 2.12 would not support Scala closures declared in a Scala REPL that captures anything from the REPL line objects. After this PR, such scenario is supported.

### How was this patch tested?

Added new unit test case to `org.apache.spark.repl.SingletonReplSuite`. The new test case fails without the fix in this PR, and pases with the fix.

Closes #28463 from rednaxelafx/closure-cleaner-indylambda.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-18 05:32:57 +00:00
Dongjoon Hyun 64795f9e0c
[SPARK-31743][CORE] Add spark_info metric into PrometheusResource
### What changes were proposed in this pull request?

This PR aims to add `spark_info` metric into `PrometheusResource`.

### Why are the changes needed?

This exposes Apache Spark version and revision like the following.

![Screen Shot 2020-05-17 at 6 02 20 PM](https://user-images.githubusercontent.com/9700541/82165091-990ce000-9868-11ea-82d5-8ea344eef646.png)

![Screen Shot 2020-05-17 at 6 06 32 PM](https://user-images.githubusercontent.com/9700541/82165247-2cdeac00-9869-11ea-83aa-e8083fa12a92.png)

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

Yes, but it's exposed as an additional metric.

### How was this patch tested?

Manual.
```
$ bin/spark-shell --driver-memory 4G -c spark.ui.prometheus.enabled=true

$ curl -s http://localhost:4041/metrics/executors/prometheus/ | head -n1
spark_info{version="3.1.0", revision="097d5098cca987e5f7bbb8394783c01517ebed0f"} 1.0
```

Closes #28563 from dongjoon-hyun/SPARK-31743.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-17 21:35:42 -07:00
HyukjinKwon c6d1309962
[SPARK-31742][TESTS] Increase the eventually time limit for Mino kdc in tests to fix flakiness
### What changes were proposed in this pull request?

This PR is kind of a follow up of SPARK-31631. In some cases, it only attempts once for ~35 seconds. Seems 10 seconds are not enough to try multiple times - took a quick look and seems difficult to manipulate the socket configuration as well.

It simply proposes to increase the time limit for now. It affects master and branch-3.0.

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 1 times over 34.294744142999996 seconds. Last failure message: Address already in use.
	at org.scalatest.concurrent.Eventually.tryTryAgain$1(Eventually.scala:432)
	at org.scalatest.concurrent.Eventually.eventually(Eventually.scala:439)
	at org.scalatest.concurrent.Eventually.eventually$(Eventually.scala:391)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:479)
	at org.scalatest.concurrent.Eventually.eventually(Eventually.scala:308)
	at org.scalatest.concurrent.Eventually.eventually$(Eventually.scala:307)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:479)
	at org.apache.spark.deploy.security.HadoopDelegationTokenManagerSuite.$anonfun$new$4(HadoopDelegationTokenManagerSuite.scala:106)
	at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:157)
	at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
	at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:286)
	at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
	at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:59)
	at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
	at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
	at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:59)
	at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
	at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:393)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:381)
	at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:376)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:458)
	at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
	at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
	at org.scalatest.Suite.run(Suite.scala:1124)
	at org.scalatest.Suite.run$(Suite.scala:1106)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
	at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:518)
	at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
	at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:59)
	at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
	at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
	at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
	at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:59)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:317)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:510)
	at sbt.ForkMain$Run$2.call(ForkMain.java:296)
	at sbt.ForkMain$Run$2.call(ForkMain.java:286)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: sbt.ForkMain$ForkError: java.net.BindException: Address already in use
	at sun.nio.ch.Net.bind0(Native Method)
	at sun.nio.ch.Net.bind(Net.java:433)
	at sun.nio.ch.Net.bind(Net.java:425)
	at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:223)
	at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74)
	at org.apache.mina.transport.socket.nio.NioSocketAcceptor.open(NioSocketAcceptor.java:198)
	at org.apache.mina.transport.socket.nio.NioSocketAcceptor.open(NioSocketAcceptor.java:51)
	at org.apache.mina.core.polling.AbstractPollingIoAcceptor.registerHandles(AbstractPollingIoAcceptor.java:547)
	at org.apache.mina.core.polling.AbstractPollingIoAcceptor.access$400(AbstractPollingIoAcceptor.java:68)
	at org.apache.mina.core.polling.AbstractPollingIoAcceptor$Acceptor.run(AbstractPollingIoAcceptor.java:422)
	at org.apache.mina.util.NamePreservingRunnable.run(NamePreservingRunnable.java:64)
	... 3 more
```

### Why are the changes needed?

To fix flakiness in the tests.

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

No, dev-only.

### How was this patch tested?

Jenkins will test it out.

Closes #28562 from HyukjinKwon/SPARK-31742.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-17 21:33:42 -07:00
Sungpeo Kook 097d5098cc [MINOR] Fix a typo in FsHistoryProvider loginfo
## What changes were proposed in this pull request?
a typo in logging. (just added `: `)

Closes #28505 from sungpeo/typo_fshistoryprovider.

Authored-by: Sungpeo Kook <elixir.kook@kakaocorp.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-17 09:43:01 -05:00
Max Gekk 5539ecfdac [SPARK-31725][CORE][SQL][TESTS] Set America/Los_Angeles time zone and Locale.US in tests by default
### What changes were proposed in this pull request?
Set default time zone and locale in the default constructor of `SparkFunSuite`:
- Default time zone to `America/Los_Angeles`
- Default locale to `Locale.US`

### Why are the changes needed?
1. To deduplicate code by moving common time zone and locale settings to one place SparkFunSuite
2. To have the same default time zone and locale in all tests. This should prevent errors like https://github.com/apache/spark/pull/28538

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

### How was this patch tested?
by running all affected test suites

Closes #28548 from MaxGekk/timezone-settings-SparkFunSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-17 02:26:00 +00:00
Wenchen Fan 2012d58475
[SPARK-31732][TESTS] Disable some flaky tests temporarily
### What changes were proposed in this pull request?

It's quite annoying to be blocked by flaky tests in several PRs. This PR disables them. The tests come from 3 PRs I'm recently watching:
https://github.com/apache/spark/pull/28526
https://github.com/apache/spark/pull/28463
https://github.com/apache/spark/pull/28517

### Why are the changes needed?

To make PR builder more stable

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

no

### How was this patch tested?

N/A

Closes #28547 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-16 07:33:58 -07:00
Wenchen Fan fd2d55c991 [SPARK-31405][SQL] Fail by default when reading/writing legacy datetime values from/to Parquet/Avro files
### What changes were proposed in this pull request?

When reading/writing datetime values that before the rebase switch day, from/to Avro/Parquet files, fail by default and ask users to set a config to explicitly do rebase or not.

### Why are the changes needed?

Rebase or not rebase have different behaviors and we should let users decide it explicitly. In most cases, users won't hit this exception as it only affects ancient datetime values.

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

Yes, now users will see an error when reading/writing dates before 1582-10-15 or timestamps before 1900-01-01 from/to Parquet/Avro files, with an error message to ask setting a config.

### How was this patch tested?

updated tests

Closes #28477 from cloud-fan/rebase.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-14 12:32:40 +09:00
Xingcan Cui 42951e6786 [SPARK-31632][CORE][WEBUI] Enrich the exception message when application information is unavailable
### What changes were proposed in this pull request?

This PR caught the `NoSuchElementException` and enriched the error message for `AppStatusStore.applicationInfo()` when Spark is starting up and the application information is unavailable.

### Why are the changes needed?
During the initialization of `SparkContext`, it first starts the Web UI and then set up the `LiveListenerBus` thread for dispatching the `SparkListenerApplicationStart` event (which will trigger writing the requested `ApplicationInfo` to `InMemoryStore`). If the Web UI is accessed before this info's being written to `InMemoryStore`, the following `NoSuchElementException` will be thrown.
```
 WARN org.eclipse.jetty.server.HttpChannel: /jobs/
 java.util.NoSuchElementException
 at java.util.Collections$EmptyIterator.next(Collections.java:4191)
 at org.apache.spark.util.kvstore.InMemoryStore$InMemoryIterator.next(InMemoryStore.java:467)
 at org.apache.spark.status.AppStatusStore.applicationInfo(AppStatusStore.scala:39)
 at org.apache.spark.ui.jobs.AllJobsPage.render(AllJobsPage.scala:266)
 at org.apache.spark.ui.WebUI.$anonfun$attachPage$1(WebUI.scala:89)
 at org.apache.spark.ui.JettyUtils$$anon$1.doGet(JettyUtils.scala:80)
 at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
 at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
 at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:873)
 at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1623)
 at org.apache.spark.ui.HttpSecurityFilter.doFilter(HttpSecurityFilter.scala:95)
 at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1610)
 at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:540)
 at org.eclipse.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:255)
 at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1345)
 at org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
 at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
 at org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
 at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
 at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
 at org.eclipse.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:753)
 at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:220)
 at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
 at org.eclipse.jetty.server.Server.handle(Server.java:505)
 at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:370)
 at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
 at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
 at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:103)
 at org.eclipse.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
 at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:698)
 at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:804)
 at java.lang.Thread.run(Thread.java:748)
```

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

### How was this patch tested?
Manually tested.

This can be reproduced:

1. `./bin/spark-shell`
2. at the same time, open `http://localhost:4040/jobs/` in your browser with quickly refreshing.

Closes #28444 from xccui/SPARK-31632.

Authored-by: Xingcan Cui <xccui@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-14 12:07:22 +09:00
Kousuke Saruta 7952f44dac
[SPARK-31697][WEBUI] HistoryServer should set Content-Type
### What changes were proposed in this pull request?

This PR changes HistoryServer to set Content-Type.

I noticed that we will get html as plain text when we access to wrong URLs which represent non-existence appId on HistoryServer.

```
<html>
      <head>
        <meta http-equiv="Content-type" content="text/html; charset=utf-8"/><meta name="viewport" content="width=device-width, initial-scale=1"/><link rel="stylesheet" href="/static/bootstrap.min.css" type="text/css"/><link rel="stylesheet" href="/static/vis-timeline-graph2d.min.css" type="text/css"/><link rel="stylesheet" href="/static/webui.css" type="text/css"/><link rel="stylesheet" href="/static/timeline-view.css" type="text/css"/><script src="/static/sorttable.js"></script><script src="/static/jquery-3.4.1.min.js"></script><script src="/static/vis-timeline-graph2d.min.js"></script><script src="/static/bootstrap.bundle.min.js"></script><script src="/static/initialize-tooltips.js"></script><script src="/static/table.js"></script><script src="/static/timeline-view.js"></script><script src="/static/log-view.js"></script><script src="/static/webui.js"></script><script>setUIRoot('')</script>

        <link rel="shortcut icon" href="/static/spark-logo-77x50px-hd.png"></link>
        <title>Not Found</title>
      </head>
      <body>
        <div class="container-fluid">
          <div class="row">
            <div class="col-12">
              <h3 style="vertical-align: middle; display: inline-block;">
                <a style="text-decoration: none" href="/">
                  <img src="/static/spark-logo-77x50px-hd.png"/>
                  <span class="version" style="margin-right: 15px;">3.1.0-SNAPSHOT</span>
                </a>
                Not Found
              </h3>
            </div>
          </div>
          <div class="row">
            <div class="col-12">
              <div class="row">Application local-1589239 not found.</div>
            </div>
          </div>
        </div>
      </body>
    </html>
```
The reason is Content-Type not set. I confirmed it with `curl -I http://localhost:18080/history/<wrong-appId>`
```
HTTP/1.1 404 Not Found
Date: Wed, 13 May 2020 06:59:29 GMT
Cache-Control: no-cache, no-store, must-revalidate
X-Frame-Options: SAMEORIGIN
X-XSS-Protection: 1; mode=block
X-Content-Type-Options: nosniff
Content-Length: 1778
Server: Jetty(9.4.18.v20190429)
```

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

I added a test case for this issue.

Closes #28519 from sarutak/fix-content-type.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-13 01:46:40 -07:00
Kousuke Saruta 38bc45b0b5
[SPARK-30654][WEBUI][FOLLOWUP] Remove bootstrap-tooltip.js which is no longer used
### What changes were proposed in this pull request?

This PR removes `bootstrap-tooltip.js` which is no longer used.
That script is replaced with `bootstrap.bundle.min.js` in SPARK-30654 ( #27370 ).

### Why are the changes needed?

For cleaning up repository..

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

No.

### How was this patch tested?

Manually checked whether tooltips are shown in the UI and no error message shown in the debug console.

Closes #28515 from sarutak/remove-tooltipjs.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-13 01:04:57 -07:00
Dongjoon Hyun 07209f3e2d [SPARK-31683][CORE] Make Prometheus output consistent with DropWizard 4.1 result
### What changes were proposed in this pull request?

This PR aims to update Prometheus-related output format to be consistent with DropWizard 4.1 result.
- Add `Number` metrics for gauges metrics.
- Add `type` labels.

### Why are the changes needed?

SPARK-29032 added Prometheus support. After that, SPARK-29674 upgraded DropWizard for JDK9+ support and this caused difference in output labels and number of keys for Guage metrics. The current status is different from Apache Spark 2.4.5. Since we cannot change DropWizard, this PR aims to be consistent in Apache Spark 3.0.0 only.

**DropWizard 3.x**
```
metrics_master_aliveWorkers_Value 1.0
```

**DropWizard 4.1**
```
metrics_master_aliveWorkers_Value{type="gauges",} 1.0
metrics_master_aliveWorkers_Number{type="gauges",} 1.0
```

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

Yes, but this is a new feature in 3.0.0.

### How was this patch tested?

Manually check the output like the following.

**JMXExporter Result**
```
$ curl -s http://localhost:8088/ | grep "^metrics_master" | sort
metrics_master_aliveWorkers_Number{type="gauges",} 1.0
metrics_master_aliveWorkers_Value{type="gauges",} 1.0
metrics_master_apps_Number{type="gauges",} 0.0
metrics_master_apps_Value{type="gauges",} 0.0
metrics_master_waitingApps_Number{type="gauges",} 0.0
metrics_master_waitingApps_Value{type="gauges",} 0.0
metrics_master_workers_Number{type="gauges",} 1.0
metrics_master_workers_Value{type="gauges",} 1.0
```

**This PR**
```
$ curl -s http://localhost:8080/metrics/master/prometheus/ | grep master
metrics_master_aliveWorkers_Number{type="gauges"} 1
metrics_master_aliveWorkers_Value{type="gauges"} 1
metrics_master_apps_Number{type="gauges"} 0
metrics_master_apps_Value{type="gauges"} 0
metrics_master_waitingApps_Number{type="gauges"} 0
metrics_master_waitingApps_Value{type="gauges"} 0
metrics_master_workers_Number{type="gauges"} 1
metrics_master_workers_Value{type="gauges"} 1
```

Closes #28510 from dongjoon-hyun/SPARK-31683.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-05-12 19:57:48 +00:00
oleg d7c3e9e53e
[SPARK-31456][CORE] Fix shutdown hook priority edge cases
### What changes were proposed in this pull request?
Fix application order for shutdown hooks for the priorities of Int.MaxValue, Int.MinValue

### Why are the changes needed?
The bug causes out-of-order execution of shutdown hooks if their priorities were Int.MinValue or Int.MaxValue

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

### How was this patch tested?
Added a test covering the change.

Closes #28494 from oleg-smith/SPARK-31456_shutdown_hook_priority.

Authored-by: oleg <oleg@nexla.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-11 13:10:39 -07:00
Dongjoon Hyun b80309bdb4
[SPARK-31674][CORE][DOCS] Make Prometheus metric endpoints experimental
### What changes were proposed in this pull request?

This PR aims to new Prometheus-format metric endpoints experimental in Apache Spark 3.0.0.

### Why are the changes needed?

Although the new metrics are disabled by default, we had better make it experimental explicitly in Apache Spark 3.0.0 since the output format is still not fixed. We can finalize it in Apache Spark 3.1.0.

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

Only doc-change is visible to the users.

### How was this patch tested?

Manually check the code since this is a documentation and class annotation change.

Closes #28495 from dongjoon-hyun/SPARK-31674.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-10 22:32:26 -07:00
Kousuke Saruta 0fb607ef37 [SPARK-30385][WEBUI] WebUI occasionally throw IOException on stop()
### What changes were proposed in this pull request?

This PR added a workaround for the issue which occasionally happens when SparkContext#stop() is called.
I think this issue can occurs on macOS with OpenJDK / OracleJDK 1.8.
If this issue happens, following stack trace is shown.
```
20/05/03 02:17:54 WARN AbstractConnector:
java.io.IOException: No such file or directory
	at sun.nio.ch.NativeThread.signal(Native Method)
	at sun.nio.ch.ServerSocketChannelImpl.implCloseSelectableChannel(ServerSocketChannelImpl.java:292)
	at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:234)
	at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:115)
	at org.eclipse.jetty.server.ServerConnector.close(ServerConnector.java:368)
	at org.eclipse.jetty.server.AbstractNetworkConnector.shutdown(AbstractNetworkConnector.java:105)
	at org.eclipse.jetty.server.Server.doStop(Server.java:439)
	at org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:89)
	at org.apache.spark.ui.ServerInfo.stop(JettyUtils.scala:501)
	at org.apache.spark.ui.WebUI.$anonfun$stop$2(WebUI.scala:173)
	at org.apache.spark.ui.WebUI.$anonfun$stop$2$adapted(WebUI.scala:173)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.ui.WebUI.stop(WebUI.scala:173)
	at org.apache.spark.ui.SparkUI.stop(SparkUI.scala:101)
	at org.apache.spark.SparkContext.$anonfun$stop$6(SparkContext.scala:1966)
	at org.apache.spark.SparkContext.$anonfun$stop$6$adapted(SparkContext.scala:1966)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.SparkContext.$anonfun$stop$5(SparkContext.scala:1966)
	at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1357)
	at org.apache.spark.SparkContext.stop(SparkContext.scala:1966)
	at org.apache.spark.repl.Main$.$anonfun$doMain$3(Main.scala:79)
	at org.apache.spark.repl.Main$.$anonfun$doMain$3$adapted(Main.scala:79)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.repl.Main$.doMain(Main.scala:79)
	at org.apache.spark.repl.Main$.main(Main.scala:58)
	at org.apache.spark.repl.Main.main(Main.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

This issue happens when the Jetty's acceptor thread shrinks before the main thread sends a signal to the thread.

Jetty's acceptor thread waits for a new connection request and blocked by `accept(this.fd, newfd, isaa)` in [`sun.nio.ch.ServerSocketChannelImpl#accept`](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l241).

When `org.eclipse.jetty.server.Server.doStop` is called in the main thread, the thread reaches [this code](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l280).

The server socket descriptor will be closed by `nd.preClose` in the main thread.
Then, `accept()` in acceptor thread throws an Exception due to "Bad file descriptor" in case of macOS.
After the exception is thrown, the acceptor thread will continue to [fetch a task](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L783).
If the thread obtain the `SHRINK` task [here](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L854), the thread will be shrink.
If, the acceptor thread finishes before `NativeThread.signal` is called in the main thread, this issue happens.

I have confirmed this issue happens even `jetty-9.4.28.v20200408`.
Because the stack trace is displayed by the [logger](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-server/src/main/java/org/eclipse/jetty/server/ServerConnector.java#L372), it's difficult to suppress it.
According to [this condition](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L842), shrink doesn't happen if the idle time is 0. So this PR adds a workaround that set the idle time to 0 immediately before stop.

In case of Linux, the acceptor thread is still blocked by `accept` even though `np.preClose` is called in the main thread.
The acceptor thread will return from `accept` when `NativeThread.signal` is called in the main thread.
It seems that the implementation of `accept systemcall` called in `accept` is different between Linux and macOS.
So, I believe this issue doesn't happen on Linux.

Also, the implementation of `NativeThread.signal` is a little bit changed in [OpenJDK 9](http://hg.openjdk.java.net/jdk9/jdk9/jdk/rev/7b17bff2ea36) for macOS.
So this issue doesn't happen for macOS with OpenJDK 9+.

You can reproduce this issue by following instructions using debugger.

1. Launch spark-shell in local mode with JDWP enabled.
2. Access to WebUI. This is needed to increase the number of SparkUI thread to greater than minThreads to meet the condition of shrink.
3. Enable the following breakpoints. Note that don't suspend all threads when a thread reaches one of the breakpoints. Only the threads which reach the line should be suspended.
  3.1 [long now = System.nanoTime(); at org.eclipse.jetty.util.thread.QueuedThreadPool#idleJobPoll](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L850)
  3.2 [NativeThread.signal(th); at sun.nio.ch.ServerSocketChannelImpl#implCloseSelectableChannel](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l283)
  3.3 [thread = 0; at ServerSocketChannelImpl#accept](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l247)
4. Quit spark-shell.
5.  Waiting for a thread reaching the breakpoint `3.1` and until the following condition become true (The idle time of those threads are 1min and you can confirm it using the expression evaluation feature if your debugger supports ).
`(System.nanoTime() - last) > TimeUnit.MILLISECONDS.toNanos(_idleTimeout)`
6. The acceptor thread named `SparkUI-<N>-acceptor-0` should be suspended at the breakpoint `3.3` so continue this thread. This thread will reach the breakpoint at `3.1` and continue further. Then, the acceptor thread will be shrink.
7. Continue all the threads rest.

### Why are the changes needed?

This stack trace is not brought by Spark but it confuses users.

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

No.

### How was this patch tested?

Tested by the reproduce procedure above and confirmed acceptor thread is no longer shrink.

Closes #28437 from sarutak/SPARK-30385.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-08 08:41:18 +00:00
Dongjoon Hyun 24fac1e0c7
[SPARK-31646][FOLLOWUP][TESTS] Add clean up code and disable irrelevent conf 2020-05-07 17:50:32 -07:00
tianlzhang dad61ed465
[SPARK-31646][SHUFFLE] Remove unused registeredConnections counter from ShuffleMetrics
### What changes were proposed in this pull request?
Remove unused `registeredConnections` counter from `ExternalBlockHandler#ShuffleMetrics`

This was added by SPARK-25642 at 3.0.0
- 8dd29fe36b

### Why are the changes needed?
It's `registeredConnections` counter created in `TransportContext` that's really counting the numbers and it's misleading for people who want to add new metrics like `registeredConnections`.

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

### How was this patch tested?
Add UTs to ensure all expected metrics are registered for `ExternalShuffleService` and `YarnShuffleService`

Closes #28457 from manuzhang/spark-31611-pre.

Lead-authored-by: tianlzhang <tianlzhang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-07 15:22:13 -07:00
Kent Yao bd6b53cc0b [SPARK-31631][TESTS] Fix test flakiness caused by MiniKdc which throws 'address in use' BindException with retry
### What changes were proposed in this pull request?
The `Kafka*Suite`s are flaky because of the Hadoop MiniKdc issue - https://issues.apache.org/jira/browse/HADOOP-12656
> Looking at MiniKdc implementation, if port is 0, the constructor use ServerSocket to find an unused port, assign the port number to the member variable port and close the ServerSocket object; later, in initKDCServer(), instantiate a TcpTransport object and bind at that port.

> It appears that the port may be used in between, and then throw the exception.

Related test failures are suspected,  such as https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/122225/testReport/org.apache.spark.sql.kafka010/KafkaDelegationTokenSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/

```scala
[info] org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** (15 seconds, 426 milliseconds)
[info]   java.net.BindException: Address already in use
[info]   at sun.nio.ch.Net.bind0(Native Method)
[info]   at sun.nio.ch.Net.bind(Net.java:433)
[info]   at sun.nio.ch.Net.bind(Net.java:425)
[info]   at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:223)
[info]   at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74)
[info]   at org.apache.mina.transport.socket.nio.NioSocketAcceptor.open(NioSocketAcceptor.java:198)
[info]   at org.apache.mina.transport.socket.nio.NioSocketAcceptor.open(NioSocketAcceptor.java:51)
[info]   at org.apache.mina.core.polling.AbstractPollingIoAcceptor.registerHandles(AbstractPollingIoAcceptor.java:547)
[info]   at org.apache.mina.core.polling.AbstractPollingIoAcceptor.access$400(AbstractPollingIoAcceptor.java:68)
[info]   at org.apache.mina.core.polling.AbstractPollingIoAcceptor$Acceptor.run(AbstractPollingIoAcceptor.java:422)
[info]   at org.apache.mina.util.NamePreservingRunnable.run(NamePreservingRunnable.java:64)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:748)
```
After comparing the error stack trace with similar issues reported  in different projects, such as
https://issues.apache.org/jira/browse/KAFKA-3453
https://issues.apache.org/jira/browse/HBASE-14734

We can be sure that they are caused by the same problem issued in HADOOP-12656.

In the PR, We apply the approach from HBASE first before we finally drop Hadoop 2.7.x

### Why are the changes needed?

fix test flakiness

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

### How was this patch tested?

the test itself passing Jenkins

Closes #28442 from yaooqinn/SPARK-31631.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-07 14:37:03 +09:00
zhengruifeng ebdf41dd69 [SPARK-30642][ML][PYSPARK] LinearSVC blockify input vectors
### What changes were proposed in this pull request?
1, add new param `blockSize`;
2, add a new class InstanceBlock;
3, **if `blockSize==1`, keep original behavior; if `blockSize>1`, stack input vectors to blocks (like ALS/MLP);**
4, if `blockSize>1`, standardize the input outside of optimization procedure;

### Why are the changes needed?
1, reduce RAM to persist traing dataset; (save about 40% RAM)
2, use Level-2 BLAS routines; (4x ~ 5x faster on dataset `epsilon`)

### Does this PR introduce any user-facing change?
Yes, a new param is added

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

Closes #28349 from zhengruifeng/blockify_svc_II.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2020-05-06 10:06:23 +08:00
yi.wu 61a6ca5d3f
[SPARK-31643][TEST] Fix flaky o.a.s.scheduler.BarrierTaskContextSuite.barrier task killed, interrupt
### What changes were proposed in this pull request?

Make sure the task has nearly reached `context.barrier()` before killing.

### Why are the changes needed?

In case of the task is killed before it reaches `context.barrier()`, the task will not create the expected file.

```
Error Message
org.scalatest.exceptions.TestFailedException: new java.io.File(dir, killedFlagFile).exists() was false Expect barrier task being killed.
Stacktrace
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: new java.io.File(dir, killedFlagFile).exists() was false Expect barrier task being killed.
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$testBarrierTaskKilled$1(BarrierTaskContextSuite.scala:266)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$testBarrierTaskKilled$1$adapted(BarrierTaskContextSuite.scala:226)
	at org.apache.spark.SparkFunSuite.withTempDir(SparkFunSuite.scala:163)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.testBarrierTaskKilled(BarrierTaskContextSuite.scala:226)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$new$29(BarrierTaskContextSuite.scala:277)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
```

[Here's](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/122273/testReport/org.apache.spark.scheduler/BarrierTaskContextSuite/barrier_task_killed__interrupt/) the full error messages.

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

### How was this patch tested?

Closes #28454 from Ngone51/fix_kill_interrupt.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-05 12:36:42 -07:00
Akshat Bordia c71198ab6c [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver
### What changes were proposed in this pull request?
Fixing an issue where Spark Master UI Fails to load if the application is waiting for workers to launch driver.

**Root Cause:**
This is happening due to the fact that the submitted application is waiting for a worker to be free to run the driver. Due to this resource is set to null in the formatResourcesAddresses method and this is running into null pointer exception.
![image](https://user-images.githubusercontent.com/31816865/80801557-77ee9300-8bca-11ea-92b7-b8df58b68de3.png)

**Fix:**
Added a null check before forming a resource address and display "None" if the driver isn't launched yet.

### Why are the changes needed?

Spark Master UI should load as expected when applications are waiting for workers to run driver.

### Does this PR introduce _any_ user-facing change?
The worker column in Spark Master UI will show "None" if the driver hasn't been launched yet.
![image](https://user-images.githubusercontent.com/31816865/80801671-be43f200-8bca-11ea-86c3-381925f82cc7.png)

### How was this patch tested?
Tested on a local setup. Launched 2 applications and ensured that Spark Master UI loads fine.
![image](https://user-images.githubusercontent.com/31816865/80801883-5b9f2600-8bcb-11ea-8a1a-cc597aabc4c2.png)

Closes #28429 from akshatb1/MasterUIBug.

Authored-by: Akshat Bordia <akshat.bordia31@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-05-05 08:58:37 -05:00
Dongjoon Hyun 0907f2e7b5 [SPARK-27963][FOLLOW-UP][DOCS][CORE] Remove for testing because CleanerListener is used ExecutorMonitor during dynamic allocation
### What changes were proposed in this pull request?

This PR aims to remove `for testing` from `CleanerListener` class description to promote this private class more clearly.

### Why are the changes needed?

After SPARK-27963 (Allow dynamic allocation without a shuffle service), `CleanerListener` is used in `ExecutorMonitor` during dynamic allocation. Specifically, `CleanerListener.shuffleCleaned` is used.
- https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala#L385-L392

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

No.

### How was this patch tested?

This is a private doc-only change.

Closes #28452 from dongjoon-hyun/SPARK-MINOR.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-05 10:07:30 +09:00
Tianshi Zhu a222644e1d [SPARK-31267][SQL] Flaky test: WholeStageCodegenSparkSubmitSuite.Generated code on driver should not embed platform-specific constant
### What changes were proposed in this pull request?

Allow customized timeouts for `runSparkSubmit`, which will make flaky tests more likely to pass by using a larger timeout value.

I was able to reproduce the test failure on my laptop, which took 1.5 - 2 minutes to finish the test. After increasing the timeout, the test now can pass locally.

### Why are the changes needed?

This allows slow tests to use a larger timeout, so they are more likely to succeed.

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

No

### How was this patch tested?

The test was able to pass on my local env after the change.

Closes #28438 from tianshizz/SPARK-31267.

Authored-by: Tianshi Zhu <zhutianshirea@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-04 14:50:38 +09:00
Xingbo Jiang b7cde42b04 [SPARK-31619][CORE] Rename config "spark.dynamicAllocation.shuffleTimeout" to "spark.dynamicAllocation.shuffleTracking.timeout"
### What changes were proposed in this pull request?
The "spark.dynamicAllocation.shuffleTimeout" configuration only takes effect if "spark.dynamicAllocation.shuffleTracking.enabled" is true, so we should re-namespace that configuration so that it's nested under the "shuffleTracking" one.

### How was this patch tested?
Covered by current existing test cases.

Closes #28426 from jiangxb1987/confName.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 11:46:17 +09:00
Weichen Xu ee1de66fe4 [SPARK-31549][PYSPARK] Add a develop API invoking collect on Python RDD with user-specified job group
### What changes were proposed in this pull request?
I add a new API in pyspark RDD class:

def collectWithJobGroup(self, groupId, description, interruptOnCancel=False)

This API do the same thing with `rdd.collect`, but it can specify the job group when do collect.
The purpose of adding this API is, if we use:

```
sc.setJobGroup("group-id...")
rdd.collect()
```
The `setJobGroup` API in pyspark won't work correctly. This related to a bug discussed in
https://issues.apache.org/jira/browse/SPARK-31549

Note:

This PR is a rather temporary workaround for `PYSPARK_PIN_THREAD`, and as a step to migrate to  `PYSPARK_PIN_THREAD` smoothly. It targets Spark 3.0.

- `PYSPARK_PIN_THREAD` is unstable at this moment that affects whole PySpark applications.
- It is impossible to make it runtime configuration as it has to be set before JVM is launched.
- There is a thread leak issue between Python and JVM. We should address but it's not a release blocker for Spark 3.0 since the feature is experimental. I plan to handle this after Spark 3.0 due to stability.

Once `PYSPARK_PIN_THREAD` is enabled by default, we should remove this API out ideally. I will target to deprecate this API in Spark 3.1.

### Why are the changes needed?
Fix bug.

### Does this PR introduce any user-facing change?
A develop API in pyspark: `pyspark.RDD. collectWithJobGroup`

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

Closes #28395 from WeichenXu123/collect_with_job_group.

Authored-by: Weichen Xu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 10:08:16 +09:00
Kousuke Saruta 91ec2eacfa
[SPARK-31565][WEBUI] Unify the font color of label among all DAG-viz
### What changes were proposed in this pull request?

This PR unifies the font color of label as `#333333` among all DAG-viz.

### Why are the changes needed?

For the consistent appearance among all DAG-viz.
There are three types of DAG-viz in the WebUI.
One is for stages, another one is for RDDs and the last one is for query plans.
But the font color of labels are slightly different among them.

For stages, the color is `#333333` (simply 333) which is specified by `spark-dag-viz.css`.
<img width="355" alt="job-graph" src="https://user-images.githubusercontent.com/4736016/80321397-b517f580-8857-11ea-8c8e-cf68f648ab05.png">
<img width="310" alt="job-graph-color" src="https://user-images.githubusercontent.com/4736016/80321399-ba754000-8857-11ea-8708-83bdef4bc1d1.png">

For RDDs, the color is `#212529` which is specified by `bootstrap.min.js`.
<img width="386" alt="stage-graph" src="https://user-images.githubusercontent.com/4736016/80321438-f0b2bf80-8857-11ea-9c2a-13fa0fd1431c.png">
<img width="313" alt="stage-graph-color" src="https://user-images.githubusercontent.com/4736016/80321444-fa3c2780-8857-11ea-81b2-4f1203d47896.png">

For query plans, the color is `black` which is specified by `spark-sql-viz.css`.
<img width="449" alt="plan-graph" src="https://user-images.githubusercontent.com/4736016/80321490-61f27280-8858-11ea-9c3a-2c98d3d4d03b.png">
<img width="316" alt="plan-graph-color" src="https://user-images.githubusercontent.com/4736016/80321496-6ae34400-8858-11ea-8fe8-0d6e4a821608.png">

After the change, the appearance is like as follows (no change for stages).

For RDDs.
<img width="389" alt="stage-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80321613-6b300f00-8859-11ea-912f-d92474aa9f47.png">

For query plans.
<img width="456" alt="plan-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80321638-9a468080-8859-11ea-974c-33c56a8ffe1a.png">

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

Yes. The unified color is slightly lighter than ever.

### How was this patch tested?

Confirmed that the color code among all DAG-viz are `#333333` using browser's debug console.

Closes #28352 from sarutak/unify-label-color.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-26 16:57:23 -07:00
Kousuke Saruta d61c6219cd [SPARK-31534][WEBUI] Text for tooltip should be escaped
### What changes were proposed in this pull request?

This PR escapes text for tooltip for DAG Viz and Timeline View.

### Why are the changes needed?

This is a bug.
Normally, DAG Viz and Timeline View show tooltip like as follows.

<img width="278" alt="dag-viz-tooltip" src="https://user-images.githubusercontent.com/4736016/80127481-5a6c6880-85cf-11ea-8daf-cfd59aa3ba09.png">
<img width="477" alt="timeline-tooltip" src="https://user-images.githubusercontent.com/4736016/80127500-60624980-85cf-11ea-9b0f-cce301019e3a.png">

They contain a callsite properly.
However, if a callsite contains characters which should be escaped for HTML without escaping , the corresponding tooltips wouldn't show the callsite and its following text properly.
<img width="179" alt="dag-viz-tooltip-before-fixed" src="https://user-images.githubusercontent.com/4736016/80128480-b1267200-85d0-11ea-8035-ad68ae5fbcab.png">
<img width="261" alt="timeline-tooltip-before-fixed" src="https://user-images.githubusercontent.com/4736016/80128492-b5eb2600-85d0-11ea-9556-c48490110244.png">

The reason of this issue is that the source texts of the tooltip texts are not escaped.

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

No.

### How was this patch tested?

I tested manually.
First, I ran a job `sc.parallelize(1 to 10).collect` in Spark Shell then, visited AllJobsPage and JobPage and confirmed tooltip texts.
<img width="196" alt="dag-viz-tooltip-fixed" src="https://user-images.githubusercontent.com/4736016/80128813-2db95080-85d1-11ea-82f8-90a1f4547f30.png">
<img width="363" alt="timeline-tooltip-fixed" src="https://user-images.githubusercontent.com/4736016/80128824-31e56e00-85d1-11ea-9818-492b72b1c56e.png">

I also added a testcase.

Closes #28317 from sarutak/fix-tooltip.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-04-27 05:14:46 +09:00
yi.wu ab8cada1f9
[SPARK-31521][CORE] Correct the fetch size when merging blocks into a merged block
### What changes were proposed in this pull request?

Fix the wrong fetch size.

### Why are the changes needed?

The fetch size should be the sum of the size of merged block and the total size of those merging blocks. But we missed the size of merged block.

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

No.

### How was this patch tested?

Added a regression test.

Closes #28301 from Ngone51/fix_merged_block_size.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-24 22:11:35 -07:00
Holden Karau 9faad07ce7 HOTFIX Revert "[SPARK-20732][CORE] Decommission cache blocks
HOTFIX test issue introduced in SPARK-20732

Closes #28337 from holdenk/revert-SPARK-20732.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-04-24 18:51:25 -07:00
Prakhar Jain 249b214590 [SPARK-20732][CORE] Decommission cache blocks to other executors when an executor is decommissioned
### What changes were proposed in this pull request?
After changes in SPARK-20628, CoarseGrainedSchedulerBackend can decommission an executor and stop assigning new tasks on it. We should also decommission the corresponding blockmanagers in the same way. i.e. Move the cached RDD blocks from those executors to other active executors.

### Why are the changes needed?
We need to gracefully decommission the block managers so that the underlying RDD cache blocks are not lost in case the executors are taken away forcefully after some timeout (because of spotloss/pre-emptible VM etc). Its good to save as much cache data as possible.

Also In future once the decommissioning signal comes from Cluster Manager (say YARN/Mesos etc), dynamic allocation + this change gives us opportunity to downscale the executors faster by making the executors free of cache data.

Note that this is a best effort approach. We try to move cache blocks from decommissioning executors to active executors. If the active executors don't have free resources available on them for caching, then the decommissioning executors will keep the cache block which it was not able to move and it will still be able to serve them.

Current overall Flow:

1. CoarseGrainedSchedulerBackend receives a signal to decommissionExecutor. On receiving the signal, it do 2 things - Stop assigning new tasks (SPARK-20628), Send another message to BlockManagerMasterEndpoint (via BlockManagerMaster) to decommission the corresponding BlockManager.

2. BlockManagerMasterEndpoint receives "DecommissionBlockManagers" message. On receiving this, it moves the corresponding block managers to "decommissioning" state. All decommissioning BMs are excluded from the getPeers RPC call which is used for replication. All these decommissioning BMs are also sent message from BlockManagerMasterEndpoint to start decommissioning process on themselves.

3. BlockManager on worker (say BM-x) receives the "DecommissionBlockManager" message. Now it will start BlockManagerDecommissionManager thread to offload all the RDD cached blocks. This thread can make multiple reattempts to decommission the existing cache blocks (multiple reattempts might be needed as there might not be sufficient space in other active BMs initially).

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

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

Closes #27864 from prakharjain09/SPARK-20732-rddcache-1.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-04-24 11:22:08 -07:00
yi.wu 263f04db86 [SPARK-31485][CORE] Avoid application hang if only partial barrier tasks launched
### What changes were proposed in this pull request?

Use `dagScheduler.taskSetFailed` to abort a barrier stage instead of throwing exception within `resourceOffers`.

### Why are the changes needed?

Any non fatal exception thrown within Spark RPC framework can be swallowed:

100fc58da5/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala (L202-L211)

 The method `TaskSchedulerImpl.resourceOffers` is also within the scope of Spark RPC framework. Thus, throw exception inside `resourceOffers` won't fail the application.

 As a result, if a barrier stage fail the require check at `require(addressesWithDescs.size == taskSet.numTasks, ...)`, the barrier stage will fail the check again and again util all tasks from `TaskSetManager` dequeued.   But since the barrier stage isn't really executed, the application will hang.

The issue can be reproduced by the following test:

```scala
initLocalClusterSparkContext(2)
val rdd0 = sc.parallelize(Seq(0, 1, 2, 3), 2)
val dep = new OneToOneDependency[Int](rdd0)
val rdd = new MyRDD(sc, 2, List(dep), Seq(Seq("executor_h_0"),Seq("executor_h_0")))
rdd.barrier().mapPartitions { iter =>
  BarrierTaskContext.get().barrier()
  iter
}.collect()
```

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

Yes, application hang previously but fail-fast after this fix.

### How was this patch tested?

Added a regression test.

Closes #28257 from Ngone51/fix_barrier_abort.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-24 04:17:06 +00:00
bmarcott f093480af9
fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)
fix method return type doc
2020-04-23 14:38:15 -05:00
Antonin Delpeuch 497024956a
[SPARK-31518][CORE] Expose filterByRange in JavaPairRDD
### What changes were proposed in this pull request?

This exposes the `filterByRange` method from `OrderedRDDFunctions` in the Java API (as a method of JavaPairRDD).

This is the only method of `OrderedRDDFunctions` which is not exposed in the Java API so far.

### Why are the changes needed?

This improves the consistency between the Scala and Java APIs. Calling the Scala method manually from a Java context is cumbersome as it requires passing many ClassTags.

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

Yes, a new method in the Java API.

### How was this patch tested?

With unit tests. The implementation of the Scala method is already tested independently and it was not touched in this PR.

Suggesting srowen as a reviewer.

Closes #28293 from wetneb/SPARK-31518.

Authored-by: Antonin Delpeuch <antonin@delpeuch.eu>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-22 20:04:17 -07:00
Thomas Graves 95aec091e4 [SPARK-29641][PYTHON][CORE] Stage Level Sched: Add python api's and tests
### What changes were proposed in this pull request?

As part of the Stage level scheduling features, add the Python api's to set resource profiles.
This also adds the functionality to properly apply the pyspark memory configuration when specified in the ResourceProfile. The pyspark memory configuration is being passed in the task local properties. This was an easy way to get it to the PythonRunner that needs it. I modeled this off how the barrier task scheduling is passing the addresses. As part of this I added in the JavaRDD api's because those are needed by python.

### Why are the changes needed?

python api for this feature

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

Yes adds the java and python apis for user to specify a ResourceProfile to use stage level scheduling.

### How was this patch tested?

unit tests and manually tested on yarn. Tests also run to verify it errors properly on standalone and local mode where its not yet supported.

Closes #28085 from tgravescs/SPARK-29641-pr-base.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-23 10:20:39 +09:00
Nicholas Marcott 8b77b31835 [SPARK-18886][CORE][FOLLOWUP] allow follow up locality resets even if no task was launched
### What changes were proposed in this pull request?
Remove the requirement to launch a task in order to reset locality wait timer.

### Why are the changes needed?
Recently https://github.com/apache/spark/pull/27207 was merged, but contained a bug which leads to undesirable behavior.

The crux of the issue is that single resource offers couldn't reset the timer, if there had been a previous reject followed by an allResourceOffer with no available resources.
This lead to a problem where once locality level reached ANY, single resource offers are all accepted, leading allResourceOffers to be left with no resources to utilize (hence no task being launched on an all resource offer -> no timer reset). The task manager would be stuck in ANY locality level.

Noting down here the downsides of using below reset conditions, in case we want to follow up.
As this is quite complex, I could easily be missing something, so please comment/respond if you have more bad behavior scenarios or find something wrong here:
The format is:

> **Reset condition**
>  - the unwanted side effect
>      - the cause/use case

Below references to locality increase/decrease mean:
```
PROCESS_LOCAL, NODE_LOCAL ... .. ANY
    ------ locality decrease --->
   <----- locality increase -----
```

**Task launch:**
- locality decrease:
   - Blacklisting, FAIR/FIFO scheduling, or task resource requirements can minimize tasks launched
 - locality increase:
   - single task launch decreases locality despite many tasks remaining

**No delay schedule reject since last allFreeResource offer**
- locality decrease:
   - locality wait less than allFreeResource offer frequency, which occurs at least 1 per second
- locality increase:
   - single resource (or none) not rejected despite many tasks remaining (other lower priority tasks utilizing resources)

**Current impl - No delay schedule reject since last (allFreeResource offer + task launch)**
- locality decrease:
  - all from above
- locality increase:
   - single resource accepted and task launched despite many tasks remaining

The current impl is an improvement on the legacy (task launch) in that unintended locality decrease case is similar and the unintended locality increase case only occurs when the cluster is fully utilized.

For the locality increase cases, perhaps a config which specifies a certain % of tasks in a taskset to finish before resetting locality levels would be helpful.

**If** that was considered a good approach then perhaps removing the task launch as a requirement would eliminate most of downsides listed above.
Lemme know if you have more ideas for eliminating locality increase downside of **No delay schedule reject since last allFreeResource offer**

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

### How was this patch tested?
TaskSchedulerImplSuite

Also manually tested similar to how I tested in https://github.com/apache/spark/pull/27207 using [this simple app](https://github.com/bmarcott/spark-test-apps/blob/master/src/main/scala/TestLocalityWait.scala).

With the new changes, given locality wait of 10s the behavior is generally:
10 seconds of locality being respected, followed by a single full utilization of resources using ANY locality level, followed by 10 seconds of locality being respected, and so on

If the legacy flag is enabled (spark.locality.wait.legacyResetOnTaskLaunch=true), the behavior is only scheduling PROCESS_LOCAL tasks (only utilizing a single executor)

cloud-fan
tgravescs

Closes #28188 from bmarcott/nmarcott-locality-fix.

Authored-by: Nicholas Marcott <481161+bmarcott@users.noreply.github.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-04-22 08:25:24 -05:00
Liang-Chi Hsieh 1d30884963 [SPARK-31484][CORE][FLOLLOWUP] Use taskAttemptId in checkpoint filename
### What changes were proposed in this pull request?

As suggested by https://github.com/apache/spark/pull/28255#discussion_r412619438, this patch proposes to use taskAttemptId in checkpoint filename, instead of stageAttemptNumber + attemptNumber.

### Why are the changes needed?

To simplify checkpoint simplified and unique.

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

No

### How was this patch tested?

Existing tests.

Closes #28289 from viirya/SPARK-31484-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-22 21:23:48 +09:00
herman cf6038499d
[SPARK-31511][SQL] Make BytesToBytesMap iterators thread-safe
### What changes were proposed in this pull request?
This PR increases the thread safety of the `BytesToBytesMap`:
- It makes the `iterator()` and `destructiveIterator()` methods used their own `Location` object. This used to be shared, and this was causing issues when the map was being iterated over in two threads by two different iterators.
- Removes the `safeIterator()` function. This is not needed anymore.
- Improves the documentation of a couple of methods w.r.t. thread-safety.

### Why are the changes needed?
It is unexpected an iterator shares the object it is returning with all other iterators. This is a violation of the iterator contract, and it causes issues with iterators over a map that are consumed in different threads.

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

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

Closes #28286 from hvanhovell/SPARK-31511.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-21 18:17:19 -07:00
Huang-Yi a5c16cbf05
[SPARK-31233][CORE] Enhance RpcTimeoutException Log Message
### What changes were proposed in this pull request?

askAbortable method throw TimeoutException while it does no complete in time. Currently, the error message contains null as remoteAddr when receiver is in client mode.
This change is to print out correct rpcAddress instead of null in the error message.

### Why are the changes needed?

It provides the address of an endpoint which does not reply in time. It helps users to find slow executors when timeout happens.

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

No.

### How was this patch tested?

Add a unit test.

Closes #28002 from Huang-yi-3456/SPARK-31233-enhance-rpctimeoutexception-log.

Authored-by: Huang-Yi <huang.yi.3456@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-21 14:08:37 -07:00
yi.wu 7103f19fea [SPARK-31472][CORE] Make sure Barrier Task always return messages or exception with abortableRpcFuture check
### What changes were proposed in this pull request?

Rewrite the periodically check logic of  `abortableRpcFuture` to make sure that barrier task would always return either desired messages or expected exception.

This PR also simplify a bit around `AbortableRpcFuture`.

### Why are the changes needed?

Currently, the periodically check logic of  `abortableRpcFuture` is done by following:

```scala
...
var messages: Array[String] = null

while (!abortableRpcFuture.toFuture.isCompleted) {
   messages = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second)
   ...
}
return messages
```
It's possible that `abortableRpcFuture` complete before next invocation on `messages = ...`. In this case, the task may return null messages or execute successfully while it should throw exception(e.g. `SparkException` from `BarrierCoordinator`).

And here's a flaky test which caused by this bug:

```
[info] BarrierTaskContextSuite:
[info] - share messages with allGather() call *** FAILED *** (18 seconds, 705 milliseconds)
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Could not recover from a failed barrier ResultStage. Most recent failure reason: Stage failed because barrier task ResultTask(0, 2) finished unsuccessfully.
[info] java.lang.NullPointerException
[info] 	at scala.collection.mutable.ArrayOps$ofRef$.length$extension(ArrayOps.scala:204)
[info] 	at scala.collection.mutable.ArrayOps$ofRef.length(ArrayOps.scala:204)
[info] 	at scala.collection.IndexedSeqOptimized.toList(IndexedSeqOptimized.scala:285)
[info] 	at scala.collection.IndexedSeqOptimized.toList$(IndexedSeqOptimized.scala:284)
[info] 	at scala.collection.mutable.ArrayOps$ofRef.toList(ArrayOps.scala:198)
[info] 	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$new$4(BarrierTaskContextSuite.scala:68)
...
```

The test exception can be reproduced by changing the line `messages = ...` to the following:

```scala
messages = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 10.micros)
Thread.sleep(5000)
```

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

No.

### How was this patch tested?

Manually test and update some unit tests.

Closes #28245 from Ngone51/fix_barrier.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-21 10:12:56 +00:00
Onur Satici ad965103a5
[SPARK-30949][K8S][CORE] Decouple requests and parallelism on drivers in K8s
### What changes were proposed in this pull request?
`spark.driver.cores` configuration is used to set the amount of parallelism in kubernetes cluster mode drivers. Previously the amount of parallelism in the drivers were the number of cores in the host when running on JDK 8u120 or older, or the maximum of driver containers resource requests and limits when running on [JDK 8u121 or newer](https://bugs.openjdk.java.net/browse/JDK-8173345). This will enable users to specify `spark.driver.cores` to set parallelism, and specify `spark.kubernetes.driver.requests.cores` to limit the resource requests of the driver container, effectively decoupling the two

### Why are the changes needed?
Drivers submitted in kubernetes cluster mode set the parallelism of various components like `RpcEnv`, `MemoryManager`, `BlockManager` from inferring the number of available cores by calling `Runtime.getRuntime().availableProcessors()`. By using this, spark applications running on JDK 8u120 or older incorrectly get the total number of cores in the host, [ignoring the cgroup limits set by kubernetes](https://bugs.openjdk.java.net/browse/JDK-6515172). JDK 8u121 and newer runtimes do not have this problem.

Orthogonal to this, it is currently not possible to decouple resource limits on the driver container with the amount of parallelism of the various network and memory components listed above.

### Does this PR introduce any user-facing change?
Yes. Previously the amount of parallelism in kubernetes cluster mode submitted drivers were the number of cores in the host when running on JDK 8u120 or older, or the maximum of driver containers resource requests and limits when running on JDK 8u121 or newer. Now the value of `spark.driver.cores` is used.

### How was this patch tested?
happy to add tests if my proposal looks reasonable

Closes #27695 from onursatici/os/decouple-requests-and-parallelism.

Authored-by: Onur Satici <onursatici@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-20 21:32:43 -07:00
Liang-Chi Hsieh e3ac56c8f4
[SPARK-31484][CORE] Add stage attempt number to temp checkpoint filename to avoid file already existing exception
### What changes were proposed in this pull request?

To avoid file already existing exception when creating checkpoint file, this PR proposes to add stage attempt number to temporary checkpoint file.

### Why are the changes needed?

On our production clusters, we have seen checkpointing failure. The failed stage can possibly leave partial written checkpoint file, the task of retried stage to write checkpoint file could fail due to`FileAlreadyExistsException` when creating the same file, like
```
org.apache.hadoop.fs.FileAlreadyExistsException: /path_to_checkpoint/rdd-114/.part-03154-attempt-0 for client xxx.xxx.xxx.xxx already exists
org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.startFile(FSDirWriteFileOp.java:359)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2353)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2273)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:728)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:413)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:447)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:989)
	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:851)
	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:794)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2490)

	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:121)
	at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:88)
	at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:270)
	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1263)
	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1205)
	at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:473)
	at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:470)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:470)
	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:411)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:929)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:910)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:872)
	at org.apache.spark.rdd.ReliableCheckpointRDD$.writePartitionToCheckpointFile(ReliableCheckpointRDD.scala:204)
```

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

Yes. Users won't see checkpoint file already existing exception after this PR.

### How was this patch tested?

Add unit test.

Closes #28255 from viirya/delete-temp-checkpoint.

Authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-19 09:11:17 -07:00
beliefer 1513673f83 [SPARK-30913][SPARK-30841][CORE][SQL][FOLLOWUP] Supplement version information to the configuration of Tests.scala and SQL
### What changes were proposed in this pull request?
I checked all the config of Spark again. find some new commit not add version information.

**Test.scala**
Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.testing.skipValidateCores | 3.1.0 | SPARK-29154 | 474b1bb5c2bce2f83c4dd8e19b9b7c5b3aebd6c4#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  

**SQL**
Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.legacy.integerGroupingId | 3.1.0 | SPARK-30279 | 71c73d58f6e88d2558ed2e696897767d93bac60f#diff-9a6b543db706f1a90f790783d6930a13 |  

The two config only exists in branch master.

### Why are the changes needed?
Supplement version information.

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

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

Closes #28233 from beliefer/sql-conf-version-legacy-integerGroupingId.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-17 17:10:48 +09:00
yi.wu 40f9dbb628 [SPARK-31425][SQL][CORE] UnsafeKVExternalSorter/VariableLengthRowBasedKeyValueBatch should also respect UnsafeAlignedOffset
### What changes were proposed in this pull request?

Make `UnsafeKVExternalSorter` / `VariableLengthRowBasedKeyValueBatch ` also respect `UnsafeAlignedOffset` when reading the record and update some out of date comemnts.

### Why are the changes needed?

Since `BytesToBytesMap` respects `UnsafeAlignedOffset` when writing the record, `UnsafeKVExternalSorter` should also respect `UnsafeAlignedOffset` when reading the record from `BytesToBytesMap` otherwise it will causes data correctness issue.

Unlike `UnsafeKVExternalSorter` may reading records from `BytesToBytesMap`, `VariableLengthRowBasedKeyValueBatch` writes and reads records by itself. Thus, similar to #22053 and [comment](https://github.com/apache/spark/pull/22053#issuecomment-411975239) there, fix for `VariableLengthRowBasedKeyValueBatch` more likely an improvement for the support of SPARC platform.

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

No.

### How was this patch tested?

Manually tested `HashAggregationQueryWithControlledFallbackSuite` with `UAO_SIZE=8`  to simulate SPARC platform. And tests only pass with this fix.

Closes #28195 from Ngone51/fix_uao.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-17 04:48:27 +00:00
yi.wu b2e9e1717b [SPARK-31344][CORE] Polish implementation of barrier() and allGather()
### What changes were proposed in this pull request?

1. Combine  `BarrierRequestToSync` and `AllGatherRequestToSync` into `RequestToSync`, which is distinguished by `RequestMethod` type.

2. Remove unnecessary Json serialization/deserialization

3. Clean up some codes to make runBarrier() and `BarrierCoordinator` more general

4. Remove unused imports.

### Why are the changes needed?

To make codes simpler for better maintain in the future.

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

No.

### How was this patch tested?

This is pure code refactor, so should be covered by existed tests.

Closes #28117 from Ngone51/refactor_barrier.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-04-16 21:23:32 -07:00
Kousuke Saruta 8608189335
[SPARK-31446][WEBUI] Make html elements for a paged table possible to have different id attribute
### What changes were proposed in this pull request?

This PR makes each id attribute for page navigations in a page unique.

`PagedTable#pageNavigation` returns HTML elements representing a page navigation for a paged table.
In the current implementation, the method generates an id and it's used for id attribute for a set of elements for the page navigation.
But some pages have two page navigations so there are two set of elements where corresponding elements have the same id.
For example, there are two `form-completedJob-table-page` id in JobsPage.
### Why are the changes needed?

Each id attribute should be unique in a page.
The following is a screenshot of warning messages shown with Chrome when I visit JobsPage (Firefox doesn't show in my environment).
<img width="1440" alt="warning-jobspage" src="https://user-images.githubusercontent.com/4736016/79261523-f3fa9280-7eca-11ea-861d-d54f04f1b0bc.png">

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

No.

### How was this patch tested?

I added a test case for `pageNavigation` extended.
I also manually tested that there were no warning messages for the uniqueness in JobsPage and JobPage.

Closes #28217 from sarutak/unique-form-id.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-16 16:24:11 -07:00
Kousuke Saruta df27350142 [SPARK-31420][WEBUI][FOLLOWUP] Make locale of timeline-view 'en'
<!--
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.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->

### 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.
-->
This change explicitly set locale of timeline view to 'en' to be the same appearance as before upgrading vis-timeline.

### 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.
-->
We upgraded vis-timeline in #28192 and the upgraded version is different from before we used in the notation of dates.
The notation seems to be dependent on locale. The following is appearance in my Japanese environment.
<img width="557" alt="locale-changed" src="https://user-images.githubusercontent.com/4736016/79265314-de886700-7ed0-11ea-8641-fa76b993c0d9.png">

Although the notation is in Japanese, the default format is a little bit unnatural (e.g. 4月9日 05:39 is natural rather than 9 四月 05:39).

I found we can get the same appearance as before by explicitly set locale to 'en'.

### 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.
-->
I visited JobsPage, JobPage and StagePage and confirm that timeline view shows dates with 'en' locale.
<img width="735" alt="fix-date-appearance" src="https://user-images.githubusercontent.com/4736016/79267107-8bfc7a00-7ed3-11ea-8a25-f6681d04a83c.png">

NOTE: #28192 will be backported to branch-2.4 and branch-3.0 so this PR should be follow #28214 and #28213 .

Closes #28218 from sarutak/fix-locale-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-04-17 02:31:08 +09:00
Kousuke Saruta 04f04e0ea7 [SPARK-31420][WEBUI] Infinite timeline redraw in job details page
### What changes were proposed in this pull request?

Upgrade vis.js to fix an infinite re-drawing issue.

As reported here, old releases of vis.js have that issue.
Fortunately, the latest version seems to resolve the issue.

With the latest release of vis.js, there are some performance issues with the original `timeline-view.js` and `timeline-view.css` so I also changed them.

### Why are the changes needed?

For better UX.

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

No. Appearance and functionalities are not changed.

### How was this patch tested?

I confirmed infinite redrawing doesn't happen with a JobPage which I had reproduced the issue.

With the original version of vis.js, I reproduced the issue with the following conditions.

* Use history server and load core/src/test/resources/spark-events.
* Visit the JobPage for job2 in application_1553914137147_0018.
* Zoom out to 80% on Safari / Chrome / Firefox.

Maybe, it depends on OS and the version of browsers.

Closes #28192 from sarutak/upgrade-visjs.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-04-13 23:23:00 -07:00
Gengliang Wang 28e1a4fa93 [SPARK-31411][UI] Show submitted time and duration in job details page
### What changes were proposed in this pull request?

Show submitted time and duration of a job in its details page

### Why are the changes needed?

When we check job details from the SQL execution page, it will be more convenient if we can get the submission time and duration from the job page, instead of finding the info from job list page.

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

Yes. After changes, the job details page shows the submitted time and duration.

### How was this patch tested?

Manual check
![image](https://user-images.githubusercontent.com/1097932/78974997-0a1de280-7ac8-11ea-8072-ce7a001b1b0c.png)

Closes #28179 from gengliangwang/addSubmittedTimeAndDuration.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-04-13 17:12:26 -07:00
Dongjoon Hyun a6e6fbf2ca
[SPARK-31422][CORE] Fix NPE when BlockManagerSource is used after BlockManagerMaster stops
### What changes were proposed in this pull request?

This PR (SPARK-31422) aims to return empty result in order to avoid `NullPointerException` at `getStorageStatus` and `getMemoryStatus` which happens after `BlockManagerMaster` stops. The empty result is consistent with the current status of `SparkContext` because `BlockManager` and `BlockManagerMaster` is already stopped.

### Why are the changes needed?

In `SparkEnv.stop`, the following stop sequence is used and `metricsSystem.stop` invokes `sink.stop`.
```
blockManager.master.stop()
metricsSystem.stop() --> sinks.foreach(_.stop)
```

However, some sink can invoke `BlockManagerSource` and ends up with `NullPointerException` because `BlockManagerMaster` is already stopped and `driverEndpoint` became `null`.
```
java.lang.NullPointerException
at org.apache.spark.storage.BlockManagerMaster.getStorageStatus(BlockManagerMaster.scala:170)
at org.apache.spark.storage.BlockManagerSource$$anonfun$10.apply(BlockManagerSource.scala:63)
at org.apache.spark.storage.BlockManagerSource$$anonfun$10.apply(BlockManagerSource.scala:63)
at org.apache.spark.storage.BlockManagerSource$$anon$1.getValue(BlockManagerSource.scala:31)
at org.apache.spark.storage.BlockManagerSource$$anon$1.getValue(BlockManagerSource.scala:30)
```

Since `SparkContext` registers and forgets `BlockManagerSource` without deregistering, we had better avoid `NullPointerException` inside `BlockManagerMaster` preventively.
```scala
_env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager))
```

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

Yes. This will remove NPE for the users who uses `BlockManagerSource`.

### How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #28187 from dongjoon-hyun/SPARK-31422.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-11 08:27:30 -07:00
Dongjoon Hyun c6ea6933e2 [SPARK-18886][CORE][TESTS][FOLLOWUP] Fix a test failure due to InvalidUseOfMatchersException
### What changes were proposed in this pull request?

This fixes one UT failure.
```
[info] - extra resources from executor *** FAILED *** (218 milliseconds)
[info]   org.mockito.exceptions.misusing.InvalidUseOfMatchersException: Invalid use of argument matchers!
[info] 0 matchers expected, 1 recorded:
```

### Why are the changes needed?

The original PR was merged with an outdated Jenkins result (7 days before the merging).

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

No.

### How was this patch tested?

Pass the Jenkins or manually do the following.
```
$ build/sbt "core/testOnly *.CoarseGrainedSchedulerBackendSuite"
```

Closes #28174 from dongjoon-hyun/SPARK-18886.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-10 12:02:41 +09:00
Nicholas Marcott 8b4862953a [SPARK-18886][CORE] Make Locality wait time measure resource under utilization due to delay scheduling
### What changes were proposed in this pull request?

[Delay scheduling](http://elmeleegy.com/khaled/papers/delay_scheduling.pdf) is an optimization that sacrifices fairness for data locality in order to improve cluster and workload throughput.

One useful definition of "delay" here is how much time has passed since the TaskSet was using its fair share of resources.

However it is impractical to calculate this delay, as it would require running simulations assuming no delay scheduling. Tasks would be run in different orders with different run times.

Currently the heuristic used to estimate this delay is the time since a task was last launched for a TaskSet. The problem is that it essentially does not account for resource utilization, potentially leaving the cluster heavily underutilized.

This PR modifies the heuristic in an attempt to move closer to the useful definition of delay above.
The newly proposed delay is the time since a TasksSet last launched a task **and** did not reject any resources due to delay scheduling when offered its "fair share".

See the last comments of #26696 for more discussion.

### Why are the changes needed?

cluster can become heavily underutilized as described in [SPARK-18886](https://issues.apache.org/jira/browse/SPARK-18886?jql=project%20%3D%20SPARK%20AND%20text%20~%20delay)

### How was this patch tested?

TaskSchedulerImplSuite

cloud-fan
tgravescs
squito

Closes #27207 from bmarcott/nmarcott-fulfill-slots-2.

Authored-by: Nicholas Marcott <481161+bmarcott@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-09 11:00:29 +00:00
yi.wu a2789c2a51 [SPARK-31379][CORE][TEST] Fix flaky o.a.s.scheduler.CoarseGrainedSchedulerBackendSuite.extra resources from executor
### What changes were proposed in this pull request?

This PR (SPARK-31379) adds one line `when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(Seq.empty)` to avoid allocating resources.

### Why are the changes needed?

The test is flaky and here's part of error stack:

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException:
The code passed to eventually never returned normally. Attempted 325 times over 5.01070979 seconds.
Last failure message: ArrayBuffer("1", "3") did not equal Array("0", "1", "3").
...
org.apache.spark.scheduler.CoarseGrainedSchedulerBackendSuite.eventually(CoarseGrainedSchedulerBackendSuite.scala:45)
```

You can check [here](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120786/testReport/org.apache.spark.scheduler/CoarseGrainedSchedulerBackendSuite/extra_resources_from_executor/) for details.

And it is flaky because:  after sending `StatusUpdate` to `CoarseGrainedSchedulerBackend`, `CoarseGrainedSchedulerBackend` will call `makeOffer` immediately once releasing the resources. So, it's possible that `availableAddrs` has allocated again before we assert `execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")`.

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

No.

### How was this patch tested?

The issue can be stably reproduced by inserting `Thread.sleep(3000)` after the line of sending `StatusUpdate`. After applying this fix, the issue is gone.

Closes #28145 from Ngone51/fix_flaky.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-08 17:54:28 +09:00
Thomas Graves 30f1866078
[SPARK-31378][CORE] stage level scheduling dynamic allocation bug with initial num executors
### What changes were proposed in this pull request?

I found a bug in the stage level scheduling dynamic allocation code when you have a non default profile and it has an initial number of executors the same as what the number of executors needed for the first job, then we don't properly request the executors.  This causes a hang.

The issue is that when a new stage is added and the initial number of executors is set, we set the target to be the initial number.  Unfortunately that makes the code in the update and sync function think it has already requested that number.  So to fix this, when there is an initial number we just go ahead and request executors at that point. This is basically what happens on startup to handle the case with the default profile.

### Why are the changes needed?

bug

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

no

### How was this patch tested?

unit test and manually test on yarn cluster. Went though multiple scenarios initial numbers, minimum number and number executor required by the first stage.

Closes #28146 from tgravescs/SPARK-31378.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-07 14:06:19 -07:00
Holden Karau 8f010bd0a8
[SPARK-31208][CORE] Add an expiremental cleanShuffleDependencies
### What changes were proposed in this pull request?

Add a cleanShuffleDependencies as an experimental developer feature to allow folks to clean up shuffle files more aggressively than we currently do.

### Why are the changes needed?

Dynamic scaling on Kubernetes (introduced in Spark 3) depends on only shutting down executors without shuffle files. However Spark does not aggressively clean up shuffle files (see SPARK-5836) and instead depends on JVM GC on the driver to trigger deletes. We already have a mechanism to explicitly clean up shuffle files from the ALS algorithm where we create a lot of quickly orphaned shuffle files. We should expose this as an advanced developer feature to enable people to better clean-up shuffle files improving dynamic scaling of their jobs on Kubernetes.

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

This adds a new experimental API.

### How was this patch tested?

ALS already used a mechanism like this, re-targets the ALS code to the new interface, tested with existing ALS tests.

Closes #28038 from holdenk/SPARK-31208-allow-users-to-cleanup-shuffle-files.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-07 13:54:36 -07:00
Kousuke Saruta f5250a581b
[SPARK-31360][WEBUI] Fix hung-up issue in StagePage
### What changes were proposed in this pull request?

This change (SPARK-31360) fixes a hung-up issue in StagePage.
StagePage will be hung-up with following operations.

1. Run a job with shuffle.
`scala> sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _).collect`

2. Visit StagePage for the stage writing shuffle data and check `Shuffle Write Time`.
<img width="401" alt="check-shuffle-write-time" src="https://user-images.githubusercontent.com/4736016/78557730-4513e200-784c-11ea-8b42-a5053b9489a5.png">

3. Run a job with no shuffle.
`scala> sc.parallelize(1 to 10).collect`

4. Visit StagePage for the last stage.
<img width="956" alt="hungup" src="https://user-images.githubusercontent.com/4736016/78557746-4f35e080-784c-11ea-83e8-5db745b88535.png">

This issue is caused by following reason.

In stagepage.js, an array `optionalColumns` has indices for columns for optional metrics.
If a stage doesn't perform shuffle read or write, the corresponding indices are removed from the array.
StagePage doesn't try to create column for such metrics, even if the state of corresponding optional metrics are preserved as "visible".
But, if a stage doesn't perform both shuffle read and write, the index for `Shuffle Write Time` isn't removed.
In that case, StagePage tries to create a column for `Shuffle Write Time` even though there are no metrics for shuffle write, leading hungup.

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

I tested with operations I explained above and confirmed that StagePage won't be hung-up.

Closes #28136 from sarutak/fix-ui-hungup.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-06 12:38:12 -07:00
yi.wu 5d76b12e9b
[SPARK-29154][FOLLOW-UP][CORE] RDD.resourceProfile should not be serialized
### What changes were proposed in this pull request?

Mark `RDD.resourceProfile` as `transient`.

### Why are the changes needed?

`RDD.resourceProfile` should only be used at driver side.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #28108 from Ngone51/spark_29154_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-03 09:41:03 -07:00
yi.wu a4fc6a6e98 [SPARK-31249][CORE] Fix flaky CoarseGrainedSchedulerBackendSuite.custom log url for Spark UI is applied
### What changes were proposed in this pull request?

In `CoarseGrainedSchedulerBackendSuite.RegisterExecutor`, change it to post `SparkListenerExecutorAdded` before `context.reply(true)`.

### Why are the changes needed?

To fix flaky `CoarseGrainedSchedulerBackendSuite.custom log url for Spark UI is applied`.

In this test, though we use `askSync` to register executor but `askSync` could be finished before we posting the 3rd `SparkListenerExecutorAdded` event to the listener bus due to the reason that `context.reply(true)` comes before `listenerBus.post`.

The error can be reproduced if you:
- loop it for 500 times in one turn
- or, insert a `Thread.sleep(1000)` between `post` and `reply`.

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

No.

### How was this patch tested?

Loop the flaky tests for 1000 times without any error.

Closes #28100 from Ngone51/fix_spark_31249.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-03 16:05:07 +09:00
Thomas Graves 55dea9be62 [SPARK-29153][CORE] Add ability to merge resource profiles within a stage with Stage Level Scheduling
### What changes were proposed in this pull request?

For the stage level scheduling feature, add the ability to optionally merged resource profiles if they were specified on multiple RDD within a stage.  There is a config to enable this feature, its off by default (spark.scheduler.resourceProfile.mergeConflicts). When the config is set to true, Spark will merge the profiles selecting the max value of each resource (cores, memory, gpu, etc).  further documentation will be added with SPARK-30322.

This also added in the ability to check if an equivalent resource profile already exists. This is so that if a user is running stages and combining the same profiles over and over again we don't get an explosion in the number of profiles.

### Why are the changes needed?

To allow users to specify resource on multiple RDD and not worry as much about if they go into the same stage and fail.

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

Yes, when the config is turned on it now merges the profiles instead of errorring out.

### How was this patch tested?

Unit tests

Closes #28053 from tgravescs/SPARK-29153.

Lead-authored-by: Thomas Graves <tgraves@apache.org>
Co-authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-04-02 08:30:18 -05:00
turbofei ec28925236 [SPARK-31179] Fast fail the connection while last connection failed in fast fail time window
## What changes were proposed in this pull request?

For TransportFactory, the requests sent to the same address share a clientPool.
Specially, when the io.numConnectionPerPeer is 1, these requests would share a same client.
When this address is unreachable, the createClient operation would be still timeout.
And these requests would block each other during createClient, because there is a lock for this shared client.
It would cost connectionNum \* connectionTimeOut \* maxRetry to retry, and then fail the task.

It fact, it is expected that this task could fail in connectionTimeOut * maxRetry.

In this PR, I set a fastFail time window for the clientPool, if the last connection failed in this time window, the new connection would fast fail.

## Why are the changes needed?
It can save time for some cases.
## Does this PR introduce any user-facing change?
No.
## How was this patch tested?
Existing UT.

Closes #27943 from turboFei/SPARK-31179-fast-fail-connection.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-04-02 08:18:14 -05:00
ulysses 2c0e15e1d0
[SPARK-31285][CORE] uppercase schedule mode string at config
### What changes were proposed in this pull request?

In `TaskSchedulerImpl`, Spark will upper schedule mode `SchedulingMode.withName(schedulingModeConf.toUpperCase(Locale.ROOT))`.
But at other place, Spark does not. Such as [AllJobsPage](5945d46c11/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala (L304)).
We should have the same behavior and uppercase schema mode string at config.

### Why are the changes needed?

Before this pr, it's ok to set `spark.scheduler.mode=fair` logically.
But Spark will throw warn log
```
java.util.NoSuchElementException: No value found for 'fair'
	at scala.Enumeration.withName(Enumeration.scala:124)
	at org.apache.spark.ui.jobs.AllJobsPage$$anonfun$22.apply(AllJobsPage.scala:314)
	at org.apache.spark.ui.jobs.AllJobsPage$$anonfun$22.apply(AllJobsPage.scala:314)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.ui.jobs.AllJobsPage.render(AllJobsPage.scala:314)
	at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:90)
	at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:90)
	at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
```

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

Almost no.

### How was this patch tested?

Exists Test.

Closes #28049 from ulysses-you/SPARK-31285.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-01 11:46:41 -07:00
Liang-Chi Hsieh 20fc6fa839
[SPARK-31308][PYSPARK] Merging pyFiles to files argument for Non-PySpark applications
### What changes were proposed in this pull request?

This PR (SPARK-31308) proposed to add python dependencies even it is not Python applications.

### Why are the changes needed?

For now, we add `pyFiles` argument to `files` argument only for Python applications, in SparkSubmit. Like the reason in #21420, "for some Spark applications, though they're a java program, they require not only jar dependencies, but also python dependencies.", we need to add `pyFiles` to `files` even it is not Python applications.

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

Yes. After this change, for non-PySpark applications, the Python files specified by `pyFiles` are also added to `files` like PySpark applications.

### How was this patch tested?

Manually test on jupyter notebook or do `spark-submit` with `--verbose`.

```
Spark config:
...
(spark.files,file:/Users/dongjoon/PRS/SPARK-PR-28077/a.py)
(spark.submit.deployMode,client)
(spark.master,local[*])
```

Closes #28077 from viirya/pyfile.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-31 18:08:55 -07:00
yi.wu 5ec1814e22
[SPARK-31248][CORE][TEST] Fix flaky ExecutorAllocationManagerSuite.interleaving add and remove
### What changes were proposed in this pull request?

This PR (SPARK-31248) uses `ManualClock` to disable `ExecutorAllocationManager.schedule()`  in order to avoid unexpected update of target executors.

### Why are the changes needed?

`ExecutorAllocationManager` will call `schedule` periodically, which may update target executors before we checking 496f6ac860/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala (L864)

And fail the check:

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: 12 did not equal 8
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
	at org.apache.spark.ExecutorAllocationManagerSuite.$anonfun$new$51(ExecutorAllocationManagerSuite.scala:864)
	at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:151)
	at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
```

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

No.

### How was this patch tested?

Update test.

Closes #28084 from Ngone51/spark_31248.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-31 17:26:58 -07:00
Yuanjian Li 07c50784d3 [SPARK-31314][CORE] Revert SPARK-29285 to fix shuffle regression caused by creating temporary file eagerly
### What changes were proposed in this pull request?
This reverts commit 8cf76f8d61. #25962

### Why are the changes needed?
In SPARK-29285, we change to create shuffle temporary eagerly. This is helpful for not to fail the entire task in the scenario of occasional disk failure. But for the applications that many tasks don't actually create shuffle files, it caused overhead. See the below benchmark:
Env: Spark local-cluster[2, 4, 19968], each queries run 5 round, each round 5 times.
Data: TPC-DS scale=99 generate by spark-tpcds-datagen
Results:
|     | Base                                                                                        | Revert                                                                                      |
|-----|---------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------|
| Q20 | Vector(4.096865667, 2.76231748, 2.722007606, 2.514433591, 2.400373579)  Median 2.722007606  | Vector(3.763185446, 2.586498463, 2.593472842, 2.320522846, 2.224627274)  Median 2.586498463 |
| Q33 | Vector(5.872176321, 4.854397586, 4.568787136, 4.393378146, 4.423996818)  Median 4.568787136 | Vector(5.38746785, 4.361236877, 4.082311276, 3.867206824, 3.783188024)  Median 4.082311276  |
| Q52 | Vector(3.978870321, 3.225437871, 3.282411608, 2.869674887, 2.644490664)  Median 3.225437871 | Vector(4.000381522, 3.196025108, 3.248787619, 2.767444508, 2.606163423)  Median 3.196025108 |
| Q56 | Vector(6.238045133, 4.820535173, 4.609965579, 4.313509894, 4.221256227)  Median 4.609965579 | Vector(6.241611339, 4.225592467, 4.195202502, 3.757085755, 3.657525982)  Median 4.195202502 |

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

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

Closes #28072 from xuanyuanking/SPARK-29285-revert.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-31 19:01:08 +08:00
Kengo Seki 0b237bd615 [SPARK-31292][CORE][SQL] Replace toSet.toSeq with distinct for readability
### What changes were proposed in this pull request?

This PR replaces the method calls of `toSet.toSeq` with `distinct`.

### Why are the changes needed?

`toSet.toSeq` is intended to make its elements unique but a bit verbose. Using `distinct` instead is easier to understand and improves readability.

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

No

### How was this patch tested?

Tested with the existing unit tests and found no problem.

Closes #28062 from sekikn/SPARK-31292.

Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-29 08:48:08 +09:00
yi.wu 33f532a9f2
[SPARK-31259][CORE] Fix log message about fetch request size in ShuffleBlockFetcherIterator
### What changes were proposed in this pull request?

Fix incorrect log of `cureRequestSize`.

### Why are the changes needed?

In batch mode, `curRequestSize` can be the total size of several block groups. And each group should have its own request size instead of using the total size.

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

No.

### How was this patch tested?

It's only affect log.

Closes #28028 from Ngone51/fix_curRequestSize.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-26 09:11:13 -07:00
Thomas Graves 474b1bb5c2 [SPARK-29154][CORE] Update Spark scheduler for stage level scheduling
### What changes were proposed in this pull request?

This is the core scheduler changes to support Stage level scheduling.

The main changes here include modification to the DAGScheduler to look at the ResourceProfiles associated with an RDD and have those applied inside the scheduler.
Currently if multiple RDD's in a stage have conflicting ResourceProfiles we throw an error. logic to allow this will happen in SPARK-29153. I added the interfaces to RDD to add and get the REsourceProfile so that I could add unit tests for the scheduler. These are marked as private for now until we finish the feature and will be exposed in SPARK-29150. If you think this is confusing I can remove those and remove the tests and add them back later.
I modified the task scheduler to make sure to only schedule on executor that exactly match the resource profile. It will then check those executors to make sure the current resources meet the task needs before assigning it.  In here I changed the way we do the custom resource assignment.
Other changes here include having the cpus per task passed around so that we can properly account for them. Previously we just used the one global config, but now it can change based on the ResourceProfile.
I removed the exceptions that require the cores to be the limiting resource. With this change all the places I found that used executor cores /task cpus as slots has been updated to use the ResourceProfile logic and look to see what resource is limiting.

### Why are the changes needed?

Stage level sheduling feature

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

No

### How was this patch tested?

unit tests and lots of manual testing

Closes #27773 from tgravescs/SPARK-29154.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-03-26 09:46:36 -05:00
Xingbo Jiang a03fbfbdd5 [SPARK-31207][CORE] Ensure the total number of blocks to fetch equals to the sum of local/hostLocal/remote blocks
### What changes were proposed in this pull request?

Assert the number of blocks to fetch equals the number of local blocks + the number of hostLocal blocks + the number of remote blocks in ShuffleBlockFetcherIterator. Also refactor the code a bit to make it easier to follow.

### Why are the changes needed?

When the numbers don't match it means something is going wrong, we should fail fast.

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

No. This is basically code refactoring.

### How was this patch tested?

Tested with existing test suites.

Closes #27972 from jiangxb1987/BlockFetcher.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-25 13:19:43 +08:00
Xingbo Jiang c2c5b2df50 [SPARK-31239][CORE][TEST] Increase await duration in WorkerDecommissionSuite.verify a task with all workers decommissioned succeeds
### What changes were proposed in this pull request?

The test case has been flaky because the execution time sometimes exceeds the await duration. Increase the await duration to avoid flakiness.

### How was this patch tested?

Tested locally and it didn't fail anymore.

Closes #28007 from jiangxb1987/DecomTest.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-25 13:43:35 +09:00
Kousuke Saruta 88864c0615 [SPARK-31161][WEBUI] Refactor the on-click timeline action in streagming-page.js
### What changes were proposed in this pull request?

Refactor `streaming-page.js` by making on-click timeline action customizable.

### Why are the changes needed?

In the current implementation, `streaming-page.js` is used from Streaming page and Structured Streaming page but the implementation of the on-click timeline action is strongly dependent on Streamng page.
Structured Streaming page doesn't define the on-click action for now but it's better to remove the dependncy for the future.

Originally, I make this change to fix `SPARK-31128` but #27883 resolved it.
So, now this is just for refactoring.

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

No.

### How was this patch tested?

Manual tests with following code and confirmed there are no regression and no error in the debug console in Firefox.

For Structured Streaming:
```
spark.readStream.format("socket").options(Map("host"->"localhost", "port"->"8765")).load.writeStream.format("console").start
```
And then, visited Structured Streaming page and there were no error in the debug console when I clicked a point in the timeline.

For Spark Streaming:
```
import org.apache.spark.streaming._
val ssc = new StreamingContext(sc, Seconds(1))
ssc.socketTextStream("localhost", 8765)
dstream.foreachRDD(rdd => rdd.foreach(println))
ssc.start
```
And then, visited Streaming page and confirmed scrolling down and hilighting work well and there were no error in the debug console when I clicked a point in the timeline.

Closes #27921 from sarutak/followup-SPARK-29543-fix-oncick.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-03-24 13:00:46 -05:00
beliefer ae0699d4b5 [SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core
### What changes were proposed in this pull request?
This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 |
spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |  
spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 |
spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |  
spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |  
spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |  
spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 |
spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |  
spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |  
spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 |
spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 |
spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 |
spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |  
spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |  
spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |  
spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |  
spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |  
spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |  
spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |  
spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |  
spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |  
spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |  
spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 |
spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 |
spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 |
spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |  
spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |  
spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |  
spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |  
spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d |
spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |  
spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |  
spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |  
spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |  
spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |  
spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |  
spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |  
spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |  
spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |  
spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 |
spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |  
spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27931 from beliefer/add-version-to-core-config-part-four.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 11:07:43 +09:00
sarthfrey-db 6fd3138e9c [SPARK-30667][CORE] Change BarrierTaskContext allGather method return type
This PR proposes that we change the return type of the `BarrierTaskContext.allGather` method to `Array[String]` instead of `ArrayBuffer[String]` since it is immutable. Based on discussion in #27640. cc zhengruifeng srowen

Closes #27951 from sarthfrey/all-gather-api.

Authored-by: sarthfrey-db <sarth.frey@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-19 12:12:39 +09:00
Adam Binford 9f27a5495d
[SPARK-30860][CORE] Use FileSystem.mkdirs to avoid umask at rolling event log folder and appStatusFile creation
### What changes were proposed in this pull request?
This pull request fixes an issue with rolling event logs. The rolling event log directory is created ignoring the dfs umask setting. This allows the history server to prune old rolling logs when run as the group owner of the event log folder.

### Why are the changes needed?
For non-rolling event logs, log files are created ignoring the umask setting by calling setPermission after creating the file. The default umask of 022 currently causes rolling log directories to be created without group write permissions, preventing the history server from pruning logs of applications not run as the same user as the history server. This adds the same behavior for rolling event logs so users don't need to worry about the umask setting causing different behavior.

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

### How was this patch tested?
Manually. The folder is created with the correct 770 permission. The status file is still affected by the umask setting, but that doesn't stop the folder from being deleted by the history server. I'm not sure if that causes any other issues. I'm not sure how to test something involving a Hadoop setting.

Closes #27764 from Kimahriman/bug/rolling-log-permissions.

Authored-by: Adam Binford <adam.binford@radiantsolutions.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-17 11:20:10 -07:00
Pedro Rossi ed06d98044
[SPARK-25355][K8S] Add proxy user to driver if present on spark-submit
### What changes were proposed in this pull request?

This PR adds the proxy user on the spark-submit command to the childArgs, so the proxy user can be retrieved and used in the KubernetesAplication to add the proxy user in the driver container args

### Why are the changes needed?

The proxy user when used on the spark submit doesn't work on the Kubernetes environment since it doesn't add the `--proxy-user` argument on the driver container and when I added it manually to the Pod definition it worked just fine.

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

No.

### How was this patch tested?

Tests were added

Closes #27422 from PedroRossi/SPARK-25355.

Authored-by: Pedro Rossi <pgrr@cin.ufpe.br>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-16 21:53:58 -07:00
LantaoJin 08bdc9c9b2 [SPARK-31068][SQL] Avoid IllegalArgumentException in broadcast exchange
### What changes were proposed in this pull request?
Fix the IllegalArgumentException in broadcast exchange when numRows over 341 million but less than 512 million.

Since the maximum number of keys that `BytesToBytesMap` supports is 1 << 29, and only 70% of the slots can be used before growing in `HashedRelation`, So here the limitation should not be greater equal than 341 million (1 << 29 / 1.5(357913941)) instead of 512 million.

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

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

Closes #27828 from LantaoJin/SPARK-31068.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Alan Jin <jinlantao@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-03-15 20:20:23 -05:00
beliefer f4cd7495f1 [SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core
### What changes were proposed in this pull request?
This PR follows up #27847 and https://github.com/apache/spark/pull/27852.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 |
spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |  
spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |  
spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |  
spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |  
spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |  
spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49 |  
spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |  
spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |  
spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |  
spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT
spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |  
spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |  
spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |  
spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |  
spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |
spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |  
spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |  
spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |  
spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |  
spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 |
spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |  
spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |  
spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 |
spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |  
spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |  
spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |  
spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |  
spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e |
spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT
spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |  
spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a |
spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |  
spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |  
spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |  
spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |  
spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |  
spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b |
spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |  
spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |  
spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |  
spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27913 from beliefer/add-version-to-core-config-part-three.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-16 10:08:07 +09:00
Dale Clarke 2a4fed0443 [SPARK-30654][WEBUI] Bootstrap4 WebUI upgrade
### What changes were proposed in this pull request?
Spark's Web UI is using an older version of Bootstrap (v. 2.3.2) for the portal pages. Bootstrap 2.x was moved to EOL in Aug 2013 and Bootstrap 3.x was moved to EOL in July 2019 (https://github.com/twbs/release). Older versions of Bootstrap are also getting flagged in security scans for various CVEs:

https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-72889
https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-173700
https://snyk.io/vuln/npm:bootstrap:20180529
https://snyk.io/vuln/npm:bootstrap:20160627

I haven't validated each CVE, but it would be nice to resolve any potential issues and get on a supported release.

The bad news is that there have been quite a few changes between Bootstrap 2 and Bootstrap 4. I've tried updating the library, refactoring/tweaking the CSS and JS to maintain a similar appearance and functionality, and testing the UI for functionality and appearance. This is a fairly large change so I'm sure additional testing and fixes will be needed.

### How was this patch tested?
This has been manually tested, but there is a ton of functionality and there are many pages and detail pages so it is very possible bugs introduced from the upgrade were missed. Additional testing and feedback is welcomed. If it appears a whole page was missed let me know and I'll take a pass at addressing that page/section.

Closes #27370 from clarkead/bootstrap4-core-upgrade.

Authored-by: Dale Clarke <a.dale.clarke@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-13 15:24:48 -07:00
Gengliang Wang 0f463258c2 [SPARK-31128][WEBUI] Fix Uncaught TypeError in streaming statistics page
### What changes were proposed in this pull request?

There is a minor issue in https://github.com/apache/spark/pull/26201
In the streaming statistics page, there is such error
```
streaming-page.js:211 Uncaught TypeError: Cannot read property 'top' of undefined
at SVGCircleElement.<anonymous> (streaming-page.js:211)
at SVGCircleElement.__onclick (d3.min.js:1)
```
in the console after clicking the timeline graph.
![image](https://user-images.githubusercontent.com/1097932/76479745-14b26280-63ca-11ea-9079-0065321795f9.png)

This PR is to fix it.
### Why are the changes needed?

Fix the error of javascript execution.

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

No, the error shows up in the console.

### How was this patch tested?

Manual test.

Closes #27883 from gengliangwang/fixSelector.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-12 20:01:17 -07:00
Gabor Somogyi 231e65092f [SPARK-30874][SQL] Support Postgres Kerberos login in JDBC connector
### What changes were proposed in this pull request?
When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.

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

In this PR I've added Postgres support (other supported databases will come in later PRs).

What this PR contains:
* Added `keytab` and `principal` JDBC options
* Added `ConnectionProvider` trait and it's impementations:
  * `BasicConnectionProvider` => unsecure connection
  * `PostgresConnectionProvider` => postgres secure connection
* Added `ConnectionProvider` tests
* Added `PostgresKrbIntegrationSuite` docker integration test
* Created `SecurityUtils` to concentrate re-usable security related functionalities
* Documentation

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

### Does this PR introduce any user-facing change?
Yes, 2 additional JDBC options added:
* keytab
* principal

If both provided then Spark does kerberos authentication.

### How was this patch tested?
To demonstrate the functionality with a standalone application I've created this repository: https://github.com/gaborgsomogyi/docker-kerberos

* Additional + existing unit tests
* Additional docker integration test
* Test on cluster manually
* `SKIP_API=1 jekyll build`

Closes #27637 from gaborgsomogyi/SPARK-30874.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-03-12 19:04:35 -07:00
Jungtaek Lim (HeartSaVioR) 3946b24328
[SPARK-31011][CORE] Log better message if SIGPWR is not supported while setting up decommission
### What changes were proposed in this pull request?

This patch changes to log better message (at least relevant to decommission) when registering signal handler for SIGPWR fails. SIGPWR is non-POSIX and not all unix-like OS support it; we can easily find the case, macOS.

### Why are the changes needed?

Spark already logs message on failing to register handler for SIGPWR, but the error message is too general which doesn't give the information of the impact. End users should be noticed that failing to register handler for SIGPWR effectively "disables" the feature of decommission.

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

No.

### How was this patch tested?

Manually tested via running standalone master/worker in macOS 10.14.6, with `spark.worker.decommission.enabled= true`, and submit an example application to run executors.

(NOTE: the message may be different a bit, as the message can be updated in review phase.)

For worker log:

```
20/03/06 17:19:13 INFO Worker: Registering SIGPWR handler to trigger decommissioning.
20/03/06 17:19:13 INFO SignalUtils: Registering signal handler for PWR
20/03/06 17:19:13 WARN SignalUtils: Failed to register SIGPWR - disabling worker decommission.
java.lang.IllegalArgumentException: Unknown signal: PWR
        at java.base/jdk.internal.misc.Signal.<init>(Signal.java:148)
        at jdk.unsupported/sun.misc.Signal.<init>(Signal.java:139)
        at org.apache.spark.util.SignalUtils$.$anonfun$registerSignal$1(SignalUtils.scala:95)
        at scala.collection.mutable.HashMap.getOrElseUpdate(HashMap.scala:86)
        at org.apache.spark.util.SignalUtils$.registerSignal(SignalUtils.scala:93)
        at org.apache.spark.util.SignalUtils$.register(SignalUtils.scala:81)
        at org.apache.spark.deploy.worker.Worker.<init>(Worker.scala:73)
        at org.apache.spark.deploy.worker.Worker$.startRpcEnvAndEndpoint(Worker.scala:887)
        at org.apache.spark.deploy.worker.Worker$.main(Worker.scala:855)
        at org.apache.spark.deploy.worker.Worker.main(Worker.scala)
```

For executor:

```
20/03/06 17:21:52 INFO CoarseGrainedExecutorBackend: Registering PWR handler.
20/03/06 17:21:52 INFO SignalUtils: Registering signal handler for PWR
20/03/06 17:21:52 WARN SignalUtils: Failed to register SIGPWR - disabling decommission feature.
java.lang.IllegalArgumentException: Unknown signal: PWR
        at java.base/jdk.internal.misc.Signal.<init>(Signal.java:148)
        at jdk.unsupported/sun.misc.Signal.<init>(Signal.java:139)
        at org.apache.spark.util.SignalUtils$.$anonfun$registerSignal$1(SignalUtils.scala:95)
        at scala.collection.mutable.HashMap.getOrElseUpdate(HashMap.scala:86)
        at org.apache.spark.util.SignalUtils$.registerSignal(SignalUtils.scala:93)
        at org.apache.spark.util.SignalUtils$.register(SignalUtils.scala:81)
        at org.apache.spark.executor.CoarseGrainedExecutorBackend.onStart(CoarseGrainedExecutorBackend.scala:86)
        at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:120)
        at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:203)
        at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
        at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
        at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
        at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
```

Closes #27832 from HeartSaVioR/SPARK-31011.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-11 20:27:00 -07:00
beliefer bd2b3f9132 [SPARK-30911][CORE][DOC] Add version information to the configuration of Status
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Status`.
2.Update the docs of `Status`.
3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |  
spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |  
spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |  
spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |  
spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT
spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |  
spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |  
spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27848 from beliefer/add-version-to-status-config.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-12 11:03:47 +09:00
beliefer c1b2675f2e [SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/27847.
I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d |
spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |  
spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |  
spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |  
spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |  
spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |  
spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |  
spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |  
spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |  
Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled
spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 |
spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 |
spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 |
spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |  
spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 |
spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27852 from beliefer/add-version-to-core-config-part-two.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-12 09:52:20 +09:00
Thomas Graves e807118eef
[SPARK-31055][DOCS] Update config docs for shuffle local host reads to have dep on external shuffle service
### What changes were proposed in this pull request?

with SPARK-27651 we now support host local reads for shuffle, but only when external shuffle service is enabled. Update the config docs to state that.

### Why are the changes needed?

clarify dependency

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

no

### How was this patch tested?

n/a

Closes #27812 from tgravescs/SPARK-27651-follow.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-09 12:17:59 -07:00
yi.wu ef51ff9dc8 [SPARK-31082][CORE] MapOutputTrackerMaster.getMapLocation should handle last mapIndex correctly
### What changes were proposed in this pull request?

In `getMapLocation`, change the condition from `...endMapIndex < statuses.length` to `...endMapIndex <= statuses.length`.

### Why are the changes needed?

`endMapIndex` is exclusive, we should include it when comparing to `statuses.length`. Otherwise, we can't get the location for last mapIndex.

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

No.

### How was this patch tested?

Updated existed test.

Closes #27850 from Ngone51/fix_getmaploction.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-09 15:53:34 +08:00
Kousuke Saruta 068bdd4415
[SPARK-31073][WEBUI] Add "shuffle write time" to task metrics summary in StagePage
### What changes were proposed in this pull request?

I've applied following changed to StagePage.
1. Added `Shuffle Write Time` to task metrics summary.
2. Added checkbox for `Shuffle Write Time` as an additional metrics.
3. Renamed `Write Time` column in task table to `Shuffle Write Time` and let it as an additional column.

### Why are the changes needed?

Task metrics summary doesn't show `Shuffle Write Time` even though it shows `Shuffle Read Blocked Time`.
`Shuffle Read Blocked Time` is let as an additional metrics so I also let `Shuffle Write Time` as an other additional metrics.

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

Yes. After this change, task metrics summary can show `Shuffle Write Time` and its visibility is controlled by a checkbox.
![additional-metrics-after](https://user-images.githubusercontent.com/4736016/76101844-677acb80-6012-11ea-9923-d95d852c775b.png)
![task-summary-after](https://user-images.githubusercontent.com/4736016/76101856-6ea1d980-6012-11ea-9670-3cf0ecd6faff.png)

`Write Time` column is already shown in task table but the title is ambiguous so I've renamed it as `Shuffle Write Time`.
After this change, this column is also additional column like `Shuffle Read Blocked Time`.
![tasks-table-after](https://user-images.githubusercontent.com/4736016/76102216-00a9e200-6013-11ea-9d51-1a6ce2abb0b9.png)

### How was this patch tested?

I've tested manually using following code and confirm the UI.
`sc.parallelize(1 to 1000).map(x => (x,x)).reduceByKey(_+_).collect`

Closes #27837 from sarutak/write-time.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-08 20:20:39 -07:00
beliefer bc490f383d [SPARK-31002][CORE][DOC] Add version information to the configuration of Core
### What changes were proposed in this pull request?
Add version information to the configuration of `Core`.
Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |  
SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath
SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions
SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath
spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory
spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |  
spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |  
SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath
spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |  
SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions
SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath
spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores
SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory
spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |  
spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |  
spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 |
spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |  
spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |
spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 |
spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |  
spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 |
spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |  
spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |  
spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27847 from beliefer/add-version-to-core-config-part-one.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-08 12:31:57 +09:00
beliefer e36227e2d9 [SPARK-30914][CORE][DOC] Add version information to the configuration of UI
### What changes were proposed in this pull request?
1.Add version information to the configuration of `UI`.
2.Update the docs of `UI`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |  
spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |  
spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |  
spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |  
spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d |
spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d |
spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 |
spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |  
spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |  
spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |  
spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |  
spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27806 from beliefer/add-version-to-UI-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-06 11:08:57 +09:00
yi.wu 8d5ef2f766 [SPARK-31052][TEST][CORE] Fix flaky test "DAGSchedulerSuite.shuffle fetch failed on speculative task, but original task succeed"
### What changes were proposed in this pull request?

This PR fix the flaky test in #27050.

### Why are the changes needed?

`SparkListenerStageCompleted` is posted by `listenerBus` asynchronously. So, we should make sure listener has consumed the event before asserting completed stages.

See [error message](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119308/testReport/org.apache.spark.scheduler/DAGSchedulerSuite/shuffle_fetch_failed_on_speculative_task__but_original_task_succeed__SPARK_30388_/):

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: List(0, 1, 1) did not equal List(0, 1, 1, 0)
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
	at org.apache.spark.scheduler.DAGSchedulerSuite.$anonfun$new$88(DAGSchedulerSuite.scala:1976)
```

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

No.

### How was this patch tested?

Update test and test locally by no failure after running hundreds of times. Note, the failure is easy to reproduce when loop running the test for hundreds of times(e.g 200)

Closes #27809 from Ngone51/fix_flaky_spark_30388.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-03-05 10:56:49 -08:00
yi.wu 2257ce2443 [SPARK-31034][CORE] ShuffleBlockFetcherIterator should always create request for last block group
### What changes were proposed in this pull request?

This is a bug fix of #27280. This PR fix the bug where `ShuffleBlockFetcherIterator` may forget to create request for the last block group.

### Why are the changes needed?

When (all blocks).sum < `targetRemoteRequestSize` and (all blocks).length > `maxBlocksInFlightPerAddress` and (last block group).size < `maxBlocksInFlightPerAddress`,
`ShuffleBlockFetcherIterator` will not create a request for the last group. Thus, it will lost data for the reduce task.

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

No.

### How was this patch tested?

Updated test.

Closes #27786 from Ngone51/fix_no_request_bug.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 21:31:26 +08:00
Yuanjian Li 7db0af5785 [SPARK-30668][SQL][FOLLOWUP] Raise exception instead of silent change for new DateFormatter
### What changes were proposed in this pull request?
This is a follow-up work for #27441. For the cases of new TimestampFormatter return null while legacy formatter can return a value, we need to throw an exception instead of silent change. The legacy config will be referenced in the error message.

### Why are the changes needed?
Avoid silent result change for new behavior in 3.0.

### Does this PR introduce any user-facing change?
Yes, an exception is thrown when we detect legacy formatter can parse the string and the new formatter return null.

### How was this patch tested?
Extend existing UT.

Closes #27537 from xuanyuanking/SPARK-30668-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 15:29:39 +08:00
Kent Yao 3edab6cc1d [MINOR][CORE] Expose the alias -c flag of --conf for spark-submit
### What changes were proposed in this pull request?

-c is short for --conf, it was introduced since v1.1.0 but hidden from users until now

### Why are the changes needed?

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

no

expose hidden feature

### How was this patch tested?

Nah

Closes #27802 from yaooqinn/conf.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-03-04 20:37:51 -08:00
beliefer ebcff675e0 [SPARK-30889][SPARK-30913][CORE][DOC] Add version information to the configuration of Tests.scala and Worker
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Tests` and `Worker`.
2.Update the docs of `Worker`.

I sorted out some information of `Tests` show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.testing.memory | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-395d07dcd46359cca610ce74357f0bb4 |  
spark.testing.dynamicAllocation.scheduleInterval | 2.3.0 | SPARK-22864 | 4e9e6aee44bb2ddb41b567d659358b22fd824222#diff-b096353602813e47074ace09a3890d56 |  
spark.testing | 1.0.1 | SPARK-1606 | ce57624b8232159fe3ec6db228afc622133df591#diff-d239aee594001f8391676e1047a0381e |  
spark.test.noStageRetry | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-6a9ff7fb74fd490a50462d45db2d5e11 |  
spark.testing.reservedMemory | 1.6.0 | SPARK-12081 | 84c44b500b5c90dffbe1a6b0aa86f01699b09b96#diff-395d07dcd46359cca610ce74357f0bb4 |
spark.testing.nHosts | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.testing.nExecutorsPerHost | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.testing.nCoresPerExecutor | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.resources.warnings.testing | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.testing.resourceProfileManager | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  

I sorted out some information of `Worker` show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.worker.resourcesFile | 3.0.0 | SPARK-27369 | 7cbe01e8efc3f6cd3a0cac4bcfadea8fcc74a955#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |  
spark.worker.timeout | 0.6.2 | None | e395aa295aeec6767df798bf1002b1f30983c1cd#diff-776a630ac2b2ec5fe85c07ca20a58fc0 |  
spark.worker.driverTerminateTimeout | 2.1.2 | SPARK-20843 | ebd72f453aa0b4f68760d28b3e93e6dd33856659#diff-829a8674171f92acd61007bedb1bfa4f |  
spark.worker.cleanup.enabled | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.cleanup.interval | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.cleanup.appDataTtl | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.preferConfiguredMasterAddress | 2.2.1 | SPARK-20529 | 75e5ea294c15ecfb7366ae15dce196aa92c87ca4#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-48ca297b6536cb92362bec1487581f05 |  
spark.worker.ui.retainedExecutors | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 |
spark.worker.ui.retainedDrivers | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 |
spark.worker.ui.compressedLogFileLengthCacheSize | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-d239aee594001f8391676e1047a0381e |  
spark.worker.decommission.enabled | 3.1.0 | SPARK-20628 | d273a2bb0fac452a97f5670edd69d3e452e3e57e#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27783 from beliefer/add-version-to-tests-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 11:58:21 +09:00
yi.wu 87b93d32a6 [SPARK-31017][TEST][CORE] Test for shuffle requests packaging with different size and numBlocks limit
### What changes were proposed in this pull request?

Added 2 tests for `ShuffleBlockFetcherIteratorSuite`.

### Why are the changes needed?

When packaging shuffle fetch requests in `ShuffleBlockFetcherIterator`, there are two limitations: `maxBytesInFlight` and `maxBlocksInFlightPerAddress`. However, we don’t have test cases to test them both, e.g. the size limitation is hit before the numBlocks limitation.

We should add test cases in `ShuffleBlockFetcherIteratorSuite` to test:

1. the size limitation is hit before the numBlocks limitation
2. the numBlocks limitation is hit before the size limitation

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

No.

### How was this patch tested?

Added new tests.

Closes #27767 from Ngone51/add_test.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-04 20:21:48 +08:00
xuesenliang 7a4cf339d7 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks
### What changes were proposed in this pull request?

When a job finished, its running (re-submitted) map stages should be marked as finished if not used by other jobs. The running tasks of these stages are cancelled.

And the ListenerBus should be notified too, otherwise, these map stage items will stay on the "Active Stages" page of web UI and never gone.

For example:

Suppose job 0 has two stages: map stage 0 and result stage 1. Map stage 0 has two partitions, and its result stage 1 has two partitions too.

**Steps to reproduce the bug:**
1. map stage 0:    start task 0(```TID 0```) and task 1 (```TID 1```), then both finished successfully.
2. result stage 1:  start task 0(```TID 2```) and task 1 (```TID 3```)
3. result stage 1:  task 0(```TID 2```) finished successfully
4. result stage 1:  speculative task 1.1(```TID 4```) launched, but then failed due to FetchFailedException.
5. driver re-submits map stage 0 and result stage 1.
6. map stage 0 (retry 1): task0(```TID 5```) launched
7. result stage 1: task 1(```TID 3```) finished successfully, so job 0 finished.
8. map stage 0 is removed from ```runningStages``` and ```stageIdToStage```, because it doesn't belong to any job.
```
  private def DAGScheduler#cleanupStateForJobAndIndependentStages(job: ActiveJob): HashSet[Stage] = {
   ...
      stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach {
        case (stageId, stage) =>
            ...
            def removeStage(stageId: Int): Unit = {
              for (stage <- stageIdToStage.get(stageId)) {
                if (runningStages.contains(stage)) {
                  logDebug("Removing running stage %d".format(stageId))
                  runningStages -= stage
                }
                ...
              }
              stageIdToStage -= stageId
            }

            jobSet -= job.jobId
            if (jobSet.isEmpty) { // no other job needs this stage
              removeStage(stageId)
            }
          }
  ...
  }

```
9. map stage 0 (retry 1): task0(TID 5) finished successfully, but its stage 0 is not in ```stageIdToStage```, so the stage not ```markStageAsFinished```
```
  private[scheduler] def DAGScheduler#handleTaskCompletion(event: CompletionEvent): Unit = {
    val task = event.task
    val stageId = task.stageId
    ...
    if (!stageIdToStage.contains(task.stageId)) {
      postTaskEnd(event)
      // Skip all the actions if the stage has been cancelled.
      return
    }
    ...
```

#### Relevant spark driver logs as follows:

```
20/01/02 11:21:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 2 output partitions
20/01/02 11:21:45 INFO DAGScheduler: Final stage: ResultStage 1 (main at NativeMethodAccessorImpl.java:0)
20/01/02 11:21:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 0)
20/01/02 11:21:45 INFO DAGScheduler: Missing parents: List(ShuffleMapStage 0)

20/01/02 11:21:45 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/01/02 11:21:45 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1))
20/01/02 11:21:45 INFO YarnClusterScheduler: Adding task set 0.0 with 2 tasks
20/01/02 11:21:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes)
20/01/02 11:21:45 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, 9.76.13.26, executor 2, partition 1, PROCESS_LOCAL, 7705 bytes)
20/01/02 11:22:18 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 32491 ms on 9.179.143.4 (executor 1) (1/2)
20/01/02 11:22:26 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 40544 ms on 9.76.13.26 (executor 2) (2/2)
20/01/02 11:22:26 INFO DAGScheduler: ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 40.854 s
20/01/02 11:22:26 INFO YarnClusterScheduler: Removed TaskSet 0.0, whose tasks have all completed, from pool

20/01/02 11:22:26 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[6] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/01/02 11:22:26 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 1 (MapPartitionsRDD[6] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1))
20/01/02 11:22:26 INFO YarnClusterScheduler: Adding task set 1.0 with 2 tasks
20/01/02 11:22:26 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 2, 9.179.143.4, executor 1, partition 0, NODE_LOCAL, 7929 bytes)
20/01/02 11:22:26 INFO TaskSetManager: Starting task 1.0 in stage 1.0 (TID 3, 9.76.13.26, executor 2, partition 1, NODE_LOCAL, 7929 bytes)
20/01/02 11:22:26 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 2) in 79 ms on 9.179.143.4 (executor 1) (1/2)

20/01/02 11:22:26 INFO TaskSetManager: Marking task 1 in stage 1.0 (on 9.76.13.26) as speculatable because it ran more than 158 ms
20/01/02 11:22:26 INFO TaskSetManager: Starting task 1.1 in stage 1.0 (TID 4, 9.179.143.52, executor 3, partition 1, ANY, 7929 bytes)
20/01/02 11:22:26 WARN TaskSetManager: Lost task 1.1 in stage 1.0 (TID 4, 9.179.143.52, executor 3): FetchFailed(BlockManagerId(1, 9.179.143.4, 7337, None), shuffleId=0, mapId=0, reduceId=1, message=org.apache.spark.shuffle.FetchFailedException: Connection reset by peer)
20/01/02 11:22:26 INFO TaskSetManager: Task 1.1 in stage 1.0 (TID 4) failed, but the task will not be re-executed (either because the task failed with a shuffle data fetch failure, so the previous stage needs to be re-run, or because a different copy of the task has already succeeded).
20/01/02 11:22:26 INFO DAGScheduler: Marking ResultStage 1 (main at NativeMethodAccessorImpl.java:0) as failed due to a fetch failure from ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0)
20/01/02 11:22:26 INFO DAGScheduler: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) failed in 0.261 s due to org.apache.spark.shuffle.FetchFailedException: Connection reset by peer
20/01/02 11:22:26 INFO DAGScheduler: Resubmitting ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) and ResultStage 1 (main at NativeMethodAccessorImpl.java:0) due to fetch failure
20/01/02 11:22:26 INFO DAGScheduler: Resubmitting failed stages

20/01/02 11:22:26 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/01/02 11:22:26 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0))
20/01/02 11:22:26 INFO YarnClusterScheduler: Adding task set 0.1 with 1 tasks
20/01/02 11:22:26 INFO TaskSetManager: Starting task 0.0 in stage 0.1 (TID 5, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes)

// NOTE: Here should be "INFO TaskSetManager: Finished task 1.0 in stage 1.0 (TID 3) in 10000 ms on 9.76.13.26 (executor 2) (2/2)"
// and this bug is being fixed in https://issues.apache.org/jira/browse/SPARK-30404

20/01/02 11:22:36 INFO TaskSetManager: Ignoring task-finished event for 1.0 in stage 1.0 because task 1 has already completed successfully

20/01/02 11:22:36 INFO YarnClusterScheduler: Removed TaskSet 1.0, whose tasks have all completed, from pool
20/01/02 11:22:36 INFO DAGScheduler: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) finished in 10.131 s
20/01/02 11:22:36 INFO DAGScheduler: Job 0 finished: main at NativeMethodAccessorImpl.java:0, took 51.031212 s

20/01/02 11:22:58 INFO TaskSetManager: Finished task 0.0 in stage 0.1 (TID 5) in 32029 ms on 9.179.143.4 (executor 1) (1/1)
20/01/02 11:22:58 INFO YarnClusterScheduler: Removed TaskSet 0.1, whose tasks have all completed, from pool
```

### Why are the changes needed?

web UI is incorrect: ```stage 0 (retry 1)``` is finished, but it stays in ```Active Stages``` Page.

![active_stage](https://user-images.githubusercontent.com/4401756/71656718-71185680-2d77-11ea-8dbc-fd8085ab3dfb.png)

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

### How was this patch tested?
A new test case is added.

And test manually on cluster. The result is as follows:
![cancel_stage](https://user-images.githubusercontent.com/4401756/71658434-04a15580-2d7f-11ea-952b-dd8dd685f37d.png)

Closes #27050 from liangxs/master.

Authored-by: xuesenliang <xuesenliang@tencent.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-03-03 09:29:43 -06:00
yi.wu b517f991fe [SPARK-30969][CORE] Remove resource coordination support from Standalone
### What changes were proposed in this pull request?

Remove automatically resource coordination support from Standalone.

### Why are the changes needed?

Resource coordination is mainly designed for the scenario where multiple workers launched on the same host. However, it's, actually, a non-existed  scenario for today's Spark. Because, Spark now can start multiple executors in a single Worker, while it only allow one executor per Worker at very beginning. So, now, it really help nothing for user to launch multiple workers on the same host. Thus, it's not worth for us to bring over complicated implementation and potential high maintain cost for such an impossible scenario.

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

No, it's Spark 3.0 feature.

### How was this patch tested?

Pass Jenkins.

Closes #27722 from Ngone51/abandon_coordination.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-03-02 11:23:07 -08:00
Gengliang Wang 6b641430c3 [SPARK-30964][CORE][WEBUI] Accelerate InMemoryStore with a new index
### What changes were proposed in this pull request?

Spark uses the class `InMemoryStore` as the KV storage for live UI and history server(by default if no LevelDB file path is provided).
In `InMemoryStore`, all the task data in one application is stored in a hashmap, which key is the task ID and the value is the task data. This fine for getting or deleting with a provided task ID.
However, Spark stage UI always shows all the task data in one stage and the current implementation is to look up all the values in the hashmap. The time complexity is O(numOfTasks).
Also, when there are too many stages (>spark.ui.retainedStages), Spark will linearly try to look up all the task data of the stages to be deleted as well.

This can be very bad for a large application with many stages and tasks. We can improve it by allowing the natural key of an entity to have a real parent index. So that on each lookup with parent node provided, Spark can look up all the natural keys(in our case, the task IDs) first, and then find the data with the natural keys in the hashmap.

### Why are the changes needed?

The in-memory KV store becomes really slow for large applications. We can improve it with a new index. The performance can be 10 times, 100 times, even 1000 times faster.
This is also possible to make the Spark driver more stable for large applications.

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

No

### How was this patch tested?

Existing unit tests.
Also, I run a benchmark with the following code
```
  val store = new InMemoryStore()
  val numberOfTasksPerStage = 10000
   (0 until 1000).map { sId =>
     (0 until numberOfTasksPerStage).map { taskId =>
       val task = newTaskData(sId * numberOfTasksPerStage + taskId, "SUCCESS", sId)
       store.write(task)
     }
   }
  val appStatusStore = new AppStatusStore(store)
  var start = System.nanoTime()
  appStatusStore.taskSummary(2, attemptId, Array(0, 0.25, 0.5, 0.75, 1))
  println("task summary run time: " + ((System.nanoTime() - start) / 1000000))
  val stageIds = Seq(1, 11, 66, 88)
  val stageKeys = stageIds.map(Array(_, attemptId))
  start = System.nanoTime()
  store.removeAllByIndexValues(classOf[TaskDataWrapper], TaskIndexNames.STAGE,
    stageKeys.asJavaCollection)
   println("clean up tasks run time: " + ((System.nanoTime() - start) / 1000000))
```

Task summary before the changes: 98642ms
Task summary after the changes: 120ms

Task clean up before the changes:  4900ms
Task clean up before the changes: 4ms

It's 800x faster after the changes in the micro-benchmark.

Closes #27716 from gengliangwang/liveUIStore.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-02 15:48:48 +08:00
beliefer 71365c2502 [SPARK-30912][CORE][DOC] Add version information to the configuration of Streaming.scala
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Streaming`.
2.Update the docs of `Streaming`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.streaming.dynamicAllocation.enabled | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.testing | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.minExecutors | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.maxExecutors | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.scalingInterval | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.scalingUpRatio | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.scalingDownRatio | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27745 from beliefer/add-version-to-streaming-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 15:16:40 +09:00
beliefer c63366a693 [SPARK-30891][CORE][DOC] Add version information to the configuration of History
### What changes were proposed in this pull request?
1.Add version information to the configuration of `History`.
2.Update the docs of `History`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.history.fs.logDirectory | 1.1.0 | SPARK-1768 | 21ddd7d1e9f8e2a726427f32422c31706a20ba3f#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.safemodeCheck.interval | 1.6.0 | SPARK-11020 | cf04fdfe71abc395163a625cc1f99ec5e54cc07e#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.update.interval | 1.4.0 | SPARK-6046 | 4527761bcd6501c362baf2780905a0018b9a74ba#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.cleaner.enabled | 1.3.0 | SPARK-3562 | 8942b522d8a3269a2a357e3a274ed4b3e66ebdde#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e | Branch branch-1.3 does not exist, exists in branch-1.4, but it is 1.3.0-SNAPSHOT in pom.xml
spark.history.fs.cleaner.interval | 1.4.0 | SPARK-5933 | 1991337336596f94698e79c2366f065c374128ab#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |
spark.history.fs.cleaner.maxAge | 1.4.0 | SPARK-5933 | 1991337336596f94698e79c2366f065c374128ab#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |
spark.history.fs.cleaner.maxNum | 3.0.0 | SPARK-28294 | bbc2be4f425c4c26450e1bf21db407e81046ce21#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.store.path | 2.3.0 | SPARK-20642 | 74daf622de4e534d5a5929b424a6e836850eefad#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.store.maxDiskUsage | 2.3.0 | SPARK-20654 | 8b497046c647a21bbed1bdfbdcb176745a1d5cd5#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.ui.port | 1.0.0 | SPARK-1276 | 9ae80bf9bd3e4da7443af97b41fe26aa5d35d70b#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.fs.inProgressOptimization.enabled | 2.4.0 | SPARK-6951 | 653fe02415a537299e15f92b56045569864b6183#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.fs.endEventReparseChunkSize | 2.4.0 | SPARK-6951 | 653fe02415a537299e15f92b56045569864b6183#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.fs.eventLog.rolling.maxFilesToRetain | 3.0.0 | SPARK-30481 | a2fe73b83c0e7c61d1c83b236565a71e3d005a71#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.eventLog.rolling.compaction.score.threshold | 3.0.0 | SPARK-30481 | a2fe73b83c0e7c61d1c83b236565a71e3d005a71#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.driverlog.cleaner.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.driverlog.cleaner.interval | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.driverlog.cleaner.maxAge | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.ui.acls.enable | 1.0.1 | Spark 1489 | c8dd13221215275948b1a6913192d40e0c8cbadd#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.ui.admin.acls | 2.1.1 | SPARK-19033 | 4ca1788805e4a0131ba8f0ccb7499ee0e0242837#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.ui.admin.acls.groups | 2.1.1 | SPARK-19033 | 4ca1788805e4a0131ba8f0ccb7499ee0e0242837#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.numReplayThreads | 2.0.0 | SPARK-13988 | 6fdd0e32a6c3fdce1f3f7e1f8d252af05c419f7b#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.retainedApplications | 1.0.0 | SPARK-1276 | 9ae80bf9bd3e4da7443af97b41fe26aa5d35d70b#diff-b49b5b9c31ddb36a9061004b5b723058 |
spark.history.provider | 1.1.0 | SPARK-1768 | 21ddd7d1e9f8e2a726427f32422c31706a20ba3f#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.kerberos.enabled | 1.0.1 | Spark-1490 | 866b03ef4d27b2160563b58d577de29ba6eb4442#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.kerberos.principal | 1.0.1 | Spark-1490 | 866b03ef4d27b2160563b58d577de29ba6eb4442#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.kerberos.keytab | 1.0.1 | Spark-1490 | 866b03ef4d27b2160563b58d577de29ba6eb4442#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.custom.executor.log.url | 3.0.0 | SPARK-26311 | ae5b2a6a92be4986ef5b8062d7fb59318cff6430#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.custom.executor.log.url.applyIncompleteApplication | 3.0.0 | SPARK-26311 | ae5b2a6a92be4986ef5b8062d7fb59318cff6430#diff-6bddeb5e25239974fc13db66266b167b |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27751 from beliefer/add-version-to-history-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 15:15:49 +09:00
beliefer 3beb4f875d [SPARK-30908][CORE][DOC] Add version information to the configuration of Kryo
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Kryo`.
2.Update the docs of `Kryo`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.kryo.registrationRequired | 1.1.0 | SPARK-2102 | efdaeb111917dd0314f1d00ee8524bed1e2e21ca#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryo.registrator | 0.5.0 | None | 91c07a33d90ab0357e8713507134ecef5c14e28a#diff-792ed56b3398163fa14e8578549d0d98 | This is not a release version, do we need to record it?
spark.kryo.classesToRegister | 1.2.0 | SPARK-1813 | 6bb56faea8d238ea22c2de33db93b1b39f492b3a#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.kryo.unsafe | 2.1.0 | SPARK-928 | bc167a2a53f5a795d089e8a884569b1b3e2cd439#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryo.pool | 3.0.0 | SPARK-26466 | 38f030725c561979ca98b2a6cc7ca6c02a1f80ed#diff-a3c6b992784f9abeb9f3047d3dcf3ed9 |  
spark.kryo.referenceTracking | 0.8.0 | None | 0a8cc309211c62f8824d76618705c817edcf2424#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryoserializer.buffer | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryoserializer.buffer.max | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-1f81c62dad0e2dfc387a974bb08c497c |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27734 from beliefer/add-version-to-kryo-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 15:14:47 +09:00
Thomas Graves 0e2ca11d80 [SPARK-29149][YARN] Update YARN cluster manager For Stage Level Scheduling
### What changes were proposed in this pull request?

Yarn side changes for Stage level scheduling.  The previous PR for dynamic allocation changes was https://github.com/apache/spark/pull/27313

Modified the data structures to store things on a per ResourceProfile basis.
 I tried to keep the code changes to a minimum, the main loop that requests just goes through each Resourceprofile and the logic inside for each one stayed very close to the same.
On submission we now have to give each ResourceProfile a separate yarn Priority because yarn doesn't support asking for containers with different resources at the same Priority. We just use the profile id as the priority level.
Using a different Priority actually makes things easier when the containers come back to match them again which ResourceProfile they were requested for.
The expectation is that yarn will only give you a container with resource amounts you requested or more. It should never give you a container if it doesn't satisfy your resource requests.

If you want to see the full feature changes you can look at https://github.com/apache/spark/pull/27053/files for reference

### Why are the changes needed?

For stage level scheduling YARN support.

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

no

### How was this patch tested?

Tested manually on YARN cluster and then unit tests.

Closes #27583 from tgravescs/SPARK-29149.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-28 15:23:33 -06:00
Thomas Graves 6c0c41fa0d [SPARK-30987][CORE] Increase the timeout on local-cluster waitUntilExecutorsUp calls
### What changes were proposed in this pull request?

The ResourceDiscoveryPlugin tests intermittently timeout. They are timing out on just bringing up the local-cluster. I am not able to reproduce locally.  I suspect the jenkins boxes are overloaded and taking longer then 10 seconds. There was another jira SPARK-29139 that increased timeout for some other of these as well. So try increasing the timeout to 60 seconds.

Examples of timeouts:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119030/testReport/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119005/testReport/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119029/testReport/

### Why are the changes needed?

tests should no longer intermittently fail.

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

no
### How was this patch tested?

unit tests ran.

Closes #27738 from tgravescs/SPARK-30987.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-28 11:43:05 -08:00
yi.wu 3b69796a89 [SPARK-30947][CORE] Log better message when accelerate resource is empty
### What changes were proposed in this pull request?

Try to log better message when accelerate resource is empty.

### Why are the changes needed?

Otherwise, it's weird to see cpu/memory resources after logging **that** resources is empty:

```
20/02/25 21:47:55 INFO ResourceUtils: ==============================================================
20/02/25 21:47:55 INFO ResourceUtils: Resources for spark.driver:

20/02/25 21:47:55 INFO ResourceUtils: ==============================================================
20/02/25 21:47:55 INFO SparkContext: Submitted application: Spark shell
20/02/25 21:47:55 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0)
20/02/25 21:47:55 INFO ResourceProfile: Limiting resource is  at -1 tasks per executor
```

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

NO.

### How was this patch tested?

Tested manually.

Closes #27693 from Ngone51/dont_log_resource.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-28 10:54:38 -08:00
Kent Yao 1383bd459a [SPARK-30970][K8S][CORE] Fix NPE while resolving k8s master url
### What changes were proposed in this pull request?

```
bin/spark-sql --master  k8s:///https://kubernetes.docker.internal:6443 --conf spark.kubernetes.container.image=yaooqinn/spark:v2.4.4
Exception in thread "main" java.lang.NullPointerException
	at org.apache.spark.util.Utils$.checkAndGetK8sMasterUrl(Utils.scala:2739)
	at org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment(SparkSubmit.scala:261)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:774)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:920)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:929)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```
Althrough `k8s:///https://kubernetes.docker.internal:6443` is a wrong master url but should not throw npe
The `case null` will never be touched.
3f4060c340/core/src/main/scala/org/apache/spark/util/Utils.scala (L2772-L2776)

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

add ut case

Closes #27721 from yaooqinn/SPARK-30970.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-28 00:01:20 -08:00
beliefer 325bf56e73 [SPARK-30888][CORE][DOC] Add version information to the configuration of Network
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Network`.
2.Update the docs of `Network`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |  
spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |  
spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |  
spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |  
spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |  
spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |  
spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |  
spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |  
spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27674 from beliefer/add-version-to-network-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 11:05:11 +09:00
beliefer c2857501d5 [SPARK-30909][CORE][DOC] Add version information to the configuration of Python
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Python`.
2.Update the docs of `Python`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.python.worker.reuse | 1.2.0 | SPARK-3030 | 2aea0da84c58a179917311290083456dfa043db7#diff-0a67bc4d171abe4df8eb305b0f4123a2 |  
spark.python.task.killTimeout | 2.2.2 | SPARK-22535 | be68f86e11d64209d9e325ce807025318f383bea#diff-0a67bc4d171abe4df8eb305b0f4123a2 |  
spark.python.use.daemon | 2.3.0 | SPARK-22554 | 57c5514de9dba1c14e296f85fb13fef23ce8c73f#diff-9008ad45db34a7eee2e265a50626841b |  
spark.python.daemon.module | 2.4.0 | SPARK-22959 | afae8f2bc82597593595af68d1aa2d802210ea8b#diff-9008ad45db34a7eee2e265a50626841b |  
spark.python.worker.module | 2.4.0 | SPARK-22959 | afae8f2bc82597593595af68d1aa2d802210ea8b#diff-9008ad45db34a7eee2e265a50626841b |  
spark.executor.pyspark.memory | 2.4.0 | SPARK-25004 | 7ad18ee9f26e75dbe038c6034700f9cd4c0e2baa#diff-6bdad48cfc34314e89599655442ff210 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27704 from beliefer/add-version-to-python-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 10:57:34 +09:00
beliefer 776e21af40 [SPARK-30910][CORE][DOC] Add version information to the configuration of R
### What changes were proposed in this pull request?
1.Add version information to the configuration of `R`.
2.Update the docs of `R`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.r.backendConnectionTimeout | 2.1.0 | SPARK-17919 | 2881a2d1d1a650a91df2c6a01275eba14a43b42a#diff-025470e1b7094d7cf4a78ea353fb3981 |  
spark.r.numRBackendThreads | 1.4.0 | SPARK-8282 | 28e8a6ea65fd08ab9cefc4d179d5c66ffefd3eb4#diff-697f7f2fc89808e0113efc71ed235db2 |  
spark.r.heartBeatInterval | 2.1.0 | SPARK-17919 | 2881a2d1d1a650a91df2c6a01275eba14a43b42a#diff-fe903bf14db371aa320b7cc516f2463c |  
spark.sparkr.r.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |  
spark.r.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27708 from beliefer/add-version-to-R-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 10:56:38 +09:00
gatorsmile 28b8713036 [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT
### What changes were proposed in this pull request?
This patch is to bump the master branch version to 3.1.0-SNAPSHOT.

### Why are the changes needed?
N/A

### Does this PR introduce any user-facing change?
N/A

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

Closes #27698 from gatorsmile/updateVersion.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-25 19:44:31 -08:00
yi.wu e9fd52282e [SPARK-30689][CORE][FOLLOW-UP] Rename config name of discovery plugin
### What changes were proposed in this pull request?

Rename config `spark.resources.discovery.plugin` to `spark.resources.discoveryPlugin`.

Also, as a side minor change: labeled `ResourceDiscoveryScriptPlugin` as `DeveloperApi` since it's not for end user.

### Why are the changes needed?

Discovery plugin doesn't need to reserve the "discovery" namespace here and it's more consistent with the interface name `ResourceDiscoveryPlugin` if we use `discoveryPlugin` instead.

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

No, it's newly added in Spark3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27689 from Ngone51/spark_30689_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 11:55:05 +09:00
Jungtaek Lim (HeartSaVioR) 9ea6c0a897
[SPARK-30943][SS] Show "batch ID" in tool tip string for Structured Streaming UI graphs
### What changes were proposed in this pull request?

This patch changes the tool tip string in Structured Streaming UI graphs to show batch ID (and timestamp as well) instead of only showing timestamp, which was a key for DStream but no longer a key for Structured Streaming.

This patch does some refactoring as there're some spots on confusion between js file for streaming and structured streaming.

Note that this patch doesn't actually change the x axis, as once we change it we should decouple the logic for graphs between streaming and structured streaming. It won't change UX meaningfully as in x axis we only show min and max which we still would like to know about "time" as well as batch ID.

### Why are the changes needed?

In Structured Streaming, everything is aligned for "batch ID" where the UI is only showing timestamp - end users have to manually find and correlate batch ID and the timestamp which is clearly a huge pain.

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

No

### How was this patch tested?

Manually tested. Screenshots:

![Screen Shot 2020-02-25 at 7 22 38 AM](https://user-images.githubusercontent.com/1317309/75197701-40b2ce80-57a2-11ea-9578-c2eb2d1091de.png)
![Screen Shot 2020-02-25 at 7 22 44 AM](https://user-images.githubusercontent.com/1317309/75197704-427c9200-57a2-11ea-9439-e0a8303d0860.png)
![Screen Shot 2020-02-25 at 7 22 58 AM](https://user-images.githubusercontent.com/1317309/75197706-43152880-57a2-11ea-9617-1276c3ba181e.png)
![Screen Shot 2020-02-25 at 7 23 04 AM](https://user-images.githubusercontent.com/1317309/75197708-43152880-57a2-11ea-9de2-7d37eaf88102.png)
![Screen Shot 2020-02-25 at 7 23 31 AM](https://user-images.githubusercontent.com/1317309/75197710-43adbf00-57a2-11ea-9ae4-4e292de39c36.png)

Closes #27687 from HeartSaVioR/SPARK-30943.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-02-25 15:29:36 -08:00
Thomas Graves c46c067f39 [SPARK-30942] Fix the warning for requiring cores to be limiting resources
### What changes were proposed in this pull request?

fix the warning for limiting resources when we don't know the number of executor cores. The issue is that there are places in the Spark code that use cores/task cpus to calculate slots and until the entire Stage level scheduling feature is in, we have to rely on the cores being the limiting resource.

Change the check to only warn when custom resources are specified.

### Why are the changes needed?

fix the check and warn when we should

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

A warning is printed

### How was this patch tested?

manually tested spark-shell with standalone mode, yarn, local mode.

Closes #27686 from tgravescs/SPARK-30942.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-25 10:55:56 -06:00
Shixiong Zhu 3126557b07 [SPARK-30936][CORE] Set FAIL_ON_UNKNOWN_PROPERTIES to false by default to parse Spark events
### What changes were proposed in this pull request?

Set `FAIL_ON_UNKNOWN_PROPERTIES` to `false` in `JsonProtocol` to allow ignore unknown fields in a Spark event. After this change, if we add new fields to a Spark event parsed by `ObjectMapper`, the event json string generated by a new Spark version can still be read by an old Spark History Server.

Since Spark History Server is an extra service, it usually takes time to upgrade, and it's possible that a Spark application is upgraded before SHS. Forwards-compatibility will allow an old SHS to support new Spark applications (may lose some new features but most of functions should still work).

### Why are the changes needed?

`JsonProtocol` is supposed to provide strong backwards-compatibility and forwards-compatibility guarantees: any version of Spark should be able to read JSON output written by any other version, including newer versions.

However, the forwards-compatibility guarantee is broken for events parsed by `ObjectMapper`. If a new field is added to an event parsed by `ObjectMapper` (e.g., 6dc5921e66 (diff-dc5c7a41fbb7479cef48b67eb41ad254R33)), the event json string generated by a new Spark version cannot be parsed by an old version of SHS right now.

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

No.

### How was this patch tested?

The new added tests.

Closes #27680 from zsxwing/SPARK-30936.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-25 12:28:31 +08:00
beliefer 7911de9d10 [SPARK-30887][CORE][DOC] Add version information to the configuration of Deploy
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Deploy`.
2.Update the docs of `Deploy`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.deploy.recoveryMode | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.recoveryMode.factory | 1.2.0 | SPARK-1830 |		deefd9d7377a8091a1d184b99066febd0e9f6afd#diff-29dffdccd5a7f4c8b496c293e87c8668 | This configuration appears in branch-1.3, but the version number in the pom.xml file corresponding to the commit is 1.2.0-SNAPSHOT
spark.deploy.recoveryDirectory | 0.8.1 | None |			d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.zookeeper.url | 0.8.1 | None |			d66c01f2b6defb3db6c1be99523b734a4d960532#diff-4457313ca662a1cd60197122d924585c |
spark.deploy.zookeeper.dir | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-a84228cb45c7d5bd93305a1f5bf720b6 |
spark.deploy.retainedApplications | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.retainedDrivers | 1.1.0 | None | 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.dead.worker.persistence | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.maxExecutorRetries | 1.6.3 | SPARK-16956 | ace458f0330f22463ecf7cbee7c0465e10fba8a8#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.spreadOut | 0.6.1 | None | bb2b9ff37cd2503cc6ea82c5dd395187b0910af0#diff-0e7ae91819fc8f7b47b0f97be7116325 |
spark.deploy.defaultCores | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-29dffdccd5a7f4c8b496c293e87c8668 |

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27668 from beliefer/add-version-to-deploy-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-25 11:39:11 +09:00
beliefer 59d6d5cbb0 [SPARK-30840][CORE][SQL] Add version property for ConfigEntry and ConfigBuilder
### What changes were proposed in this pull request?
Spark `ConfigEntry` and `ConfigBuilder` missing Spark version information of each configuration at release. This is not good for Spark user when they visiting the page of spark configuration.
http://spark.apache.org/docs/latest/configuration.html
The new Spark SQL config docs looks like:
![sql配置截屏](https://user-images.githubusercontent.com/8486025/74604522-cb882f00-50f9-11ea-8683-57a90f9e3347.png)

```
> SET -v
spark.sql.adaptive.enabled      false   When true, enable adaptive query execution.
spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin       0.2     The relation with a non-empty partition ratio lower than this config will not be considered as the build side of a broadcast-hash join in adaptive execution regardless of its size.This configuration only has an effect when 'spark.sql.adaptive.enabled' is enabled.
spark.sql.adaptive.optimizeSkewedJoin.enabled   true    When true and adaptive execution is enabled, a skewed join is automatically handled at runtime.
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor     10      A partition is considered as a skewed partition if its size is larger than this factor multiple the median partition size and also larger than  spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionMaxSplits  5       Configures the maximum number of task to handle a skewed partition in adaptive skewedjoin.
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold      64MB    Configures the minimum size in bytes for a partition that is considered as a skewed partition in adaptive skewed join.
spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled    true    Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks one by one, fetching continuous shuffle blocks for the same map task in batch can reduce IO and improve performance. Note, multiple continuous blocks exist in single fetch request only happen when 'spark.sql.adaptive.enabled' and 'spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled' is enabled, this feature also depends on a relocatable serializer, the concatenation support codec in use and the new version shuffle fetch protocol.
spark.sql.adaptive.shuffle.localShuffleReader.enabled   true    When true and 'spark.sql.adaptive.enabled' is enabled, this enables the optimization of converting the shuffle reader to local shuffle reader for the shuffle exchange of the broadcast hash join in probe side.
spark.sql.adaptive.shuffle.maxNumPostShufflePartitions  <undefined>     The advisory maximum number of post-shuffle partitions used in adaptive execution. This is used as the initial number of pre-shuffle partitions. By default it equals to spark.sql.shuffle.partitions. This configuration only has an effect when 'spark.sql.adaptive.enabled' and 'spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled' is enabled.
```

**Note**: Because there are so many configuration items that are exposed and require a lot of finishing, I will add the version numbers of these configuration items in another PR.

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27592 from beliefer/add-version-to-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-22 09:46:42 +09:00
sarthfrey-db 274b328f57 [SPARK-30667][CORE] Add all gather method to BarrierTaskContext
Fix for #27395

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

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

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

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27640 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-21 11:40:28 -08:00
Yuanjian Li a5efbb284e [SPARK-30809][SQL] Review and fix issues in SQL API docs
### What changes were proposed in this pull request?
- Add missing `since` annotation.
- Don't show classes under `org.apache.spark.sql.dynamicpruning` package in API docs.
- Fix the scope of `xxxExactNumeric` to remove it from the API docs.

### Why are the changes needed?
Avoid leaking APIs unintentionally in Spark 3.0.0.

### Does this PR introduce any user-facing change?
No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0.

### How was this patch tested?
Manually generated the API docs and verified the above issues have been fixed.

Closes #27560 from xuanyuanking/SPARK-30809.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 17:03:22 +08:00
Dongjoon Hyun fc4e56a54c [SPARK-30884][PYSPARK] Upgrade to Py4J 0.10.9
This PR aims to upgrade Py4J to `0.10.9` for better Python 3.7 support in Apache Spark 3.0.0 (master/branch-3.0). This is not for `branch-2.4`.

- Apache Spark 3.0.0 is using `Py4J 0.10.8.1` (released on 2018-10-21) because `0.10.8.1` was the first official release to support Python 3.7.
    - https://www.py4j.org/changelog.html#py4j-0-10-8-and-py4j-0-10-8-1
- `Py4J 0.10.9` was released on January 25th 2020 with better Python 3.7 support and `magic_member` bug fix.
    - https://github.com/bartdag/py4j/releases/tag/0.10.9
    - https://www.py4j.org/changelog.html#py4j-0-10-9

No.

Pass the Jenkins with the existing tests.

Closes #27641 from dongjoon-hyun/SPARK-30884.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-20 09:09:30 -08:00
HyukjinKwon 7c4ad6316e [SPARK-29148][CORE][FOLLOW-UP] Don't dynamic allocation warning when it's disabled
### What changes were proposed in this pull request?

Currently, after https://github.com/apache/spark/pull/27313, it shows the warning about dynamic allocation which is disabled by default.

```bash
$ ./bin/spark-shell
```

```
...
20/02/18 11:04:56 WARN ResourceProfile: Please ensure that the number of slots available on your executors is
limited by the number of cores to task cpus and not another custom resource. If cores is not the limiting resource
then dynamic allocation will not work properly!
```

This PR brings back the configuration checking for this warning. Seems mistakenly removed at https://github.com/apache/spark/pull/27313/files#diff-364713d7776956cb8b0a771e9b62f82dL2841

### Why are the changes needed?

To remove false warning.

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

Yes, it will don't show the warning. It's master only change so no user-facing to end users.

### How was this patch tested?

Manually tested.

Closes #27615 from HyukjinKwon/SPARK-29148.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-19 23:01:49 -08:00
Xingbo Jiang e32411eb07 Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext"
This reverts commit af63971cb7.
2020-02-19 17:04:47 -08:00
sarthfrey-db af63971cb7 [SPARK-30667][CORE] Add allGather method to BarrierTaskContext
### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

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

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27395 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
(cherry picked from commit 57254c9719)
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2020-02-19 12:10:51 -08:00
Prakhar Jain e086951349 [SPARK-30786][CORE] Fix Block replication failure propogation issue in BlockManager
### What changes were proposed in this pull request?
Currently the uploadBlockSync api in BlockTransferService always succeeds irrespective of whether the BlockManager was able to successfully replicate a block on peer block manager or not. This PR makes sure that the NettyBlockRpcServer invokes onFailure callback when it is not able to replicate the block to itself because of any reason. The onFailure callback makes sure that the BlockTransferService on client side gets the failure and retry replication the Block on some other BlockManager.

### Why are the changes needed?
Currently the Spark Block replication retry logic is not working correctly. It doesn't retry on other Block managers even when replication fails on 1 of the peers.

A user can cache an DataFrame with different replication factor. Ex - df.persist(StorageLevel.MEMORY_ONLY_2) - This will cache each partition at two different BlockManagers. When a DataFrame partition is computed first time, it is firstly stored locally on the local BlockManager and then it is replicated to other block managers based on replication factor config. The replication of block to other block managers might fail because of memory/network etc issues and so there is already provision to retry the replication on some other peer based on "spark.storage.maxReplicationFailures" config, Currently when this replication fails, the client does not know about the failure and so it doesn't retry on other peers. This PR fixes this issue.

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

### How was this patch tested?
Added Unit Test.

Closes #27539 from prakharjain09/bm_replicate.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-19 20:23:22 +08:00
yi.wu 68d7edf949 [SPARK-30812][SQL][CORE] Revise boolean config name to comply with new config naming policy
### What changes were proposed in this pull request?

Revise below config names to comply with [new config naming policy](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-naming-policy-of-Spark-configs-td28875.html):

SQL:
* spark.sql.execution.subquery.reuse.enabled / [SPARK-27083](https://issues.apache.org/jira/browse/SPARK-27083)
* spark.sql.legacy.allowNegativeScaleOfDecimal.enabled / [SPARK-30252](https://issues.apache.org/jira/browse/SPARK-30252)
* spark.sql.adaptive.optimizeSkewedJoin.enabled / [SPARK-29544](https://issues.apache.org/jira/browse/SPARK-29544)
* spark.sql.legacy.property.nonReserved / [SPARK-30183](https://issues.apache.org/jira/browse/SPARK-30183)
* spark.sql.streaming.forceDeleteTempCheckpointLocation.enabled / [SPARK-26389](https://issues.apache.org/jira/browse/SPARK-26389)
* spark.sql.analyzer.failAmbiguousSelfJoin.enabled / [SPARK-28344](https://issues.apache.org/jira/browse/SPARK-28344)
* spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled / [SPARK-30074](https://issues.apache.org/jira/browse/SPARK-30074)
* spark.sql.execution.pandas.arrowSafeTypeConversion / [SPARK-25811](https://issues.apache.org/jira/browse/SPARK-25811)
* spark.sql.legacy.looseUpcast / [SPARK-24586](https://issues.apache.org/jira/browse/SPARK-24586)
* spark.sql.legacy.arrayExistsFollowsThreeValuedLogic / [SPARK-28052](https://issues.apache.org/jira/browse/SPARK-28052)
* spark.sql.sources.ignoreDataLocality.enabled / [SPARK-29189](https://issues.apache.org/jira/browse/SPARK-29189)
* spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled / [SPARK-9853](https://issues.apache.org/jira/browse/SPARK-9853)

CORE:
* spark.eventLog.erasureCoding.enabled / [SPARK-25855](https://issues.apache.org/jira/browse/SPARK-25855)
* spark.shuffle.readHostLocalDisk.enabled / [SPARK-30235](https://issues.apache.org/jira/browse/SPARK-30235)
* spark.scheduler.listenerbus.logSlowEvent.enabled / [SPARK-29001](https://issues.apache.org/jira/browse/SPARK-29001)
* spark.resources.coordinate.enable / [SPARK-27371](https://issues.apache.org/jira/browse/SPARK-27371)
* spark.eventLog.logStageExecutorMetrics.enabled / [SPARK-23429](https://issues.apache.org/jira/browse/SPARK-23429)

### Why are the changes needed?

To comply with the config naming policy.

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

No. Configurations listed above are all newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27563 from Ngone51/revise_boolean_conf_name.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 20:39:50 +08:00
Ajith 2854091d12 [SPARK-22590][SQL] Copy sparkContext.localproperties to child thread in BroadcastExchangeExec.executionContext
### What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.exchange.BroadcastExchangeExec#relationFuture` make a copy of `org.apache.spark.SparkContext#localProperties` and pass it to the broadcast execution thread in `org.apache.spark.sql.execution.exchange.BroadcastExchangeExec#executionContext`

### Why are the changes needed?
When executing `BroadcastExchangeExec`, the relationFuture is evaluated via a separate thread. The threads inherit the `localProperties` from `sparkContext` as they are the child threads.
These threads are created in the executionContext (thread pools). Each Thread pool has a default `keepAliveSeconds` of 60 seconds for idle threads.
Scenarios where the thread pool has threads which are idle and reused for a subsequent new query, the thread local properties will not be inherited from spark context (thread properties are inherited only on thread creation) hence end up having old or no properties set. This will cause taskset properties to be missing when properties are transferred by child thread via `sparkContext.runJob/submitJob`

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

### How was this patch tested?
Added UT

Closes #27266 from ajithme/broadcastlocalprop.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 02:26:52 +08:00