Commit graph

7764 commits

Author SHA1 Message Date
yi.wu edeecada66 [SPARK-32850][CORE][K8S] Simplify the RPC message flow of decommission
### What changes were proposed in this pull request?

This PR cleans up the RPC message flow among the multiple decommission use cases, it includes changes:

* Keep `Worker`'s decommission status be consistent between the case where decommission starts from `Worker` and the case where decommission starts from the `MasterWebUI`: sending `DecommissionWorker` from `Master` to `Worker` in the latter case.

* Change from two-way communication to one-way communication when notifying decommission between driver and executor: it's obviously unnecessary for the executor to acknowledge the decommission status to the driver since the decommission request is from the driver. And it's same in reverse.

* Only send one message instead of two(`DecommissionSelf`/`DecommissionBlockManager`) when decommission the executor: executor and `BlockManager` are in the same JVM.

* Clean up codes around here.

### Why are the changes needed?

Before:

<img width="1948" alt="WeChat56c00cc34d9785a67a544dca036d49da" src="https://user-images.githubusercontent.com/16397174/92850308-dc461c80-f41e-11ea-8ac0-287825f4e0c4.png">

After:
<img width="1968" alt="WeChat05f7afb017e3f0132394c5e54245e49e" src="https://user-images.githubusercontent.com/16397174/93189571-de88dd80-f774-11ea-9300-1943920aa27d.png">

(Note the diagrams only counts those RPC calls that needed to go through the network. Local RPC calls are not counted here.)

After this change, We reduced 6 original RPC calls and added one more RPC call for keeping the consistent decommission status for the Worker. And the RPC flow becomes more clear.

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

No.

### How was this patch tested?

Updated existing tests.

Closes #29817 from Ngone51/simplify-decommission-rpc.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-23 13:58:44 +09:00
Ankit Srivastava 3819d39607 [SPARK-32998][BUILD] Add ability to override default remote repos with internal one
### What changes were proposed in this pull request?
- Building spark internally in orgs where access to outside internet is not allowed takes a long time because unsuccessful attempts are made to download artifacts from repositories which are not accessible. The unsuccessful attempts unnecessarily add significant amount of time to the build. I have seen a difference of up-to 1hr for some runs.
- Adding 1 environment variables that should be present that the start of the build and if they exist, override the default repos defined in the code and scripts.
envVariables:
      - DEFAULT_ARTIFACT_REPOSITORY=https://artifacts.internal.com/libs-release/

### Why are the changes needed?

To allow orgs to build spark internally without relying on external repositories for artifact downloads.

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

No.

### How was this patch tested?

Multiple builds with and without env variables set.

Closes #29874 from ankits/SPARK-32998.

Authored-by: Ankit Srivastava <ankit_srivastava@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-22 16:35:55 -07:00
Dongjoon Hyun a908b67502 [SPARK-33218][CORE] Update misleading log messages for removed shuffle blocks
### What changes were proposed in this pull request?

This updates the misleading log messages for removed shuffle block during migration.

### Why are the changes needed?

1. For the deleted shuffle blocks, `IndexShuffleBlockResolver` shows users WARN message saying `skipping migration`. However, `BlockManagerDecommissioner` shows users INFO message including `Migrated ShuffleBlockInfo(...)` inconsistently. Technically, we didn't migrated. We should not show `Migrated` message in this case.

```
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (2 / 3)
WARN IndexShuffleBlockResolver: Failed to resolve shuffle block ShuffleBlockInfo(109,18924), skipping migration. This is expected to occur if a block is removed after decommissioning has started.
INFO BlockManagerDecommissioner: Got migration sub-blocks List()
...
INFO BlockManagerDecommissioner: Migrated ShuffleBlockInfo(109,18924) to BlockManagerId(...)
```

2. In addition, if the shuffle file is deleted while the information is in the queue, the above messages are repeated multiple times, `spark.storage.decommission.maxReplicationFailuresPerBlock`. We had better use one line instead of the group of messages for that case.
```
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (0 / 3)
...
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (1 / 3)
...
INFO BlockManagerDecommissioner: Trying to migrate shuffle ShuffleBlockInfo(109,18924) to BlockManagerId(...) (2 / 3)
```

3. Skipping or not is a role of `BlockManagerDecommissioner` class. `IndexShuffleBlockResolver.getMigrationBlocks` is used twice differently like the following. We had better inform users at `BlockManagerDecommissioner` once.
    - At the beginning, to get the sub-blocks.
    - In case of `IOException`, to determine whether ignoring it or re-throwing. And, `BlockManagerDecommissioner` shows WARN message (`Skipping block ...`) again.

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

No. This is an update for log message info to be consistent.

### How was this patch tested?

Manually.

Closes #30129 from dongjoon-hyun/SPARK-33218.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-22 01:10:24 -07:00
Chao Sun cb3fa6c936 [SPARK-33212][BUILD] Move to shaded clients for Hadoop 3.x profile
### What changes were proposed in this pull request?

This switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x. For Hadoop 2.7, we'll still use the same modules such as hadoop-client.

In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:

```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```

which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.

Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).

### Why are the changes needed?

This serves two purposes:
- to unblock Spark from upgrading to Hadoop 3.2.2/3.3.0+. Latest Hadoop versions have upgraded to use Guava 27+ and in order to adopt the latest Hadoop versions in Spark, we'll need to resolve the Guava conflicts. This takes the approach by switching to shaded client jars provided by Hadoop.
- avoid pulling 3rd party dependencies from Hadoop and avoid potential future conflicts.

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

When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.

### How was this patch tested?

Relying on existing tests.

Closes #29843 from sunchao/SPARK-29250.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-10-22 03:21:34 +00:00
Dongjoon Hyun 7aed81d492 [SPARK-33202][CORE] Fix BlockManagerDecommissioner to return the correct migration status
### What changes were proposed in this pull request?

This PR changes `<` into `>` in the following to fix data loss during storage migrations.

```scala
// If we found any new shuffles to migrate or otherwise have not migrated everything.
- newShufflesToMigrate.nonEmpty || migratingShuffles.size < numMigratedShuffles.get()
+ newShufflesToMigrate.nonEmpty || migratingShuffles.size > numMigratedShuffles.get()
```

### Why are the changes needed?

`refreshOffloadingShuffleBlocks` should return `true` when the migration is still on-going.

Since `migratingShuffles` is defined like the following, `migratingShuffles.size > numMigratedShuffles.get()` means the migration is not finished.
```scala
// Shuffles which are either in queue for migrations or migrated
protected[storage] val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]()
```

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

No.

### How was this patch tested?

Pass the CI with the updated test cases.

Closes #30116 from dongjoon-hyun/SPARK-33202.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-21 14:37:56 -07:00
Dongjoon Hyun 385d5db941 [SPARK-33198][CORE] getMigrationBlocks should not fail at missing files
### What changes were proposed in this pull request?

This PR aims to fix `getMigrationBlocks` error handling and to add test coverage.
1. `getMigrationBlocks` should not fail at indexFile only case.
2. `assert` causes `java.lang.AssertionError` which is not an `Exception`.

### Why are the changes needed?

To handle the exception correctly.

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

No.

### How was this patch tested?

Pass the CI with the newly added test case.

Closes #30110 from dongjoon-hyun/SPARK-33198.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-20 15:02:36 -07:00
Dongjoon Hyun c824db2d8b [MINOR][CORE] Improve log message during storage decommission
### What changes were proposed in this pull request?

This PR aims to improve the log message for better analysis.

### Why are the changes needed?

Good logs are crucial always.

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

No.

### How was this patch tested?

Manual review.

Closes #30109 from dongjoon-hyun/k8s_log.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-20 14:55:08 -07:00
HyukjinKwon 2cfd215dc4 [SPARK-33191][YARN][TESTS] Fix PySpark test cases in YarnClusterSuite
### What changes were proposed in this pull request?

This PR proposes to fix:

```
org.apache.spark.deploy.yarn.YarnClusterSuite.run Python application in yarn-client mode
org.apache.spark.deploy.yarn.YarnClusterSuite.run Python application in yarn-cluster mode
org.apache.spark.deploy.yarn.YarnClusterSuite.run Python application in yarn-cluster mode using spark.yarn.appMasterEnv to override local envvar
```

it currently fails as below:

```
20/10/16 19:20:36 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0) (amp-jenkins-worker-03.amp executor 1): org.apache.spark.SparkException:
Error from python worker:
  Traceback (most recent call last):
    File "/usr/lib64/python2.6/runpy.py", line 104, in _run_module_as_main
      loader, code, fname = _get_module_details(mod_name)
    File "/usr/lib64/python2.6/runpy.py", line 79, in _get_module_details
      loader = get_loader(mod_name)
    File "/usr/lib64/python2.6/pkgutil.py", line 456, in get_loader
      return find_loader(fullname)
    File "/usr/lib64/python2.6/pkgutil.py", line 466, in find_loader
      for importer in iter_importers(fullname):
    File "/usr/lib64/python2.6/pkgutil.py", line 422, in iter_importers
      __import__(pkg)
    File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/__init__.py", line 53, in <module>
      from pyspark.rdd import RDD, RDDBarrier
    File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/rdd.py", line 34, in <module>
      from pyspark.java_gateway import local_connect_and_auth
    File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/java_gateway.py", line 29, in <module>
      from py4j.java_gateway import java_import, JavaGateway, JavaObject, GatewayParameters
    File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 60
      PY4J_TRUE = {"yes", "y", "t", "true"}
                        ^
  SyntaxError: invalid syntax
```

I think this was broken when Python 2 was dropped but was not caught because this specific test does not run when there's no change in YARN codes. See also https://github.com/apache/spark/pull/29843#issuecomment-712540024

The root cause seems like the paths are different, see https://github.com/apache/spark/pull/29843#pullrequestreview-502595199. I _think_ Jenkins uses a different Python executable via Anaconda and the executor side does not know where it is for some reasons.

This PR proposes to fix it just by explicitly specifying the absolute path for Python executable so the tests should pass in any environment.

### Why are the changes needed?

To make tests pass.

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

No, dev-only.

### How was this patch tested?

This issue looks specific to Jenkins. It should run the tests on Jenkins.

Closes #30099 from HyukjinKwon/SPARK-33191.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-21 00:31:58 +09:00
angerszhu f8277d3aa3 [SPARK-32069][CORE][SQL] Improve error message on reading unexpected directory
### What changes were proposed in this pull request?
Improve error message on reading unexpected directory

### Why are the changes needed?
Improve error message on reading unexpected directory

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

### How was this patch tested?
Ut

Closes #30027 from AngersZhuuuu/SPARK-32069.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-18 19:02:21 -07:00
Dongjoon Hyun 911dcd3983 [SPARK-33173][CORE][TESTS] Use eventually to check numOnTaskFailed in PluginContainerSuite
### What changes were proposed in this pull request?

This PR aims to use `eventually` to fix the flakiness of the test case `SPARK-33088: executor failed tasks trigger plugin calls`.

### Why are the changes needed?

The test case checks like the following.
```scala
assert(TestSparkPlugin.executorPlugin.numOnTaskStart == 2)
assert(TestSparkPlugin.executorPlugin.numOnTaskSucceeded == 0)
assert(TestSparkPlugin.executorPlugin.numOnTaskFailed == 2)
```

Although first and second passed, the third can fail.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/lastCompletedBuild/testReport/org.apache.spark.internal.plugin/PluginContainerSuite/SPARK_33088__executor_failed_tasks_trigger_plugin_calls/
- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129919/testReport/
```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: 1 did not equal 2
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
	at org.scalatest.Assertions$.newAssertionFailedException(Assertions.scala:1231)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:1295)
	at org.apache.spark.internal.plugin.PluginContainerSuite.$anonfun$new$8(PluginContainerSuite.scala:161)
```

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

No.

### How was this patch tested?

This only improves the robustness.

Closes #30072 from dongjoon-hyun/SPARK-33173.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-16 21:23:21 -07:00
Holden Karau ce6180c8c3 [SPARK-33154][CORE][K8S] Handle cleaned shuffles during migration
### What changes were proposed in this pull request?

If a block is removed between discovery to transfer fo the block, we short circuit that block and remove it from the list to transfer and increment the transferred blocks. This is complicated since both RPC errors and local read errors may be reported with the same exception class.

### Why are the changes needed?

Slow shuffle refreshes could waste time when decommissioning has already finished. Decommissioning might avoid transferring some some blocks to an otherwise live host which is marked as "full" if a deleted block fails to transfer to that host.

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

No.

### How was this patch tested?

New unit and integration tests.

Closes #30046 from holdenk/handle-cleaned-shuffles-during0migration.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-16 14:47:46 -07:00
HyukjinKwon bf52fa83b2 [SPARK-33165][SQL][TESTS][FOLLOW-UP] Use scala.Predef.assert instead
### What changes were proposed in this pull request?

This PR proposes to use `scala.Predef.assert` instead of `org.scalatest.Assertions.assert` removed at https://github.com/apache/spark/pull/30064

### Why are the changes needed?

Just to keep the same behaviour.

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

No, dev-only

### How was this patch tested?

Recover the existing asserts.

Closes #30065 from HyukjinKwon/SPARK-33165.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-16 13:50:57 +09:00
Samuel Souza 8f4fc22dc4 [SPARK-33088][CORE] Enhance ExecutorPlugin API to include callbacks on task start and end events
### What changes were proposed in this pull request?
Proposing a new set of APIs for ExecutorPlugins, to provide callbacks invoked at the start and end of each task of a job. Not very opinionated on the shape of the API, tried to be as minimal as possible for now.

### Why are the changes needed?
Changes described in detail on [SPARK-33088](https://issues.apache.org/jira/browse/SPARK-33088), but mostly this boils down to:

1. This feature was considered when the ExecutorPlugin API was initially introduced in #21923, but never implemented.
2. The use-case which **requires** this feature is to propagate tracing information from the driver to the executor, such that calls from the same job can all be traced.
  a. Tracing frameworks usually are setup in thread locals, therefore it's important for the setup to happen in the same thread which runs the tasks.
  b. Executors can be for multiple jobs, therefore it's not sufficient to set tracing information at executor startup time -- it needs to happen every time a task starts or ends.

### Does this PR introduce _any_ user-facing change?
No. This PR introduces new features for future developers to use.

### How was this patch tested?
Unit tests on `PluginContainerSuite`.

Closes #29977 from fsamuel-bs/SPARK-33088.

Authored-by: Samuel Souza <ssouza@palantir.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-10-15 22:12:41 -05:00
Takeshi Yamamuro a5c17de241 [SPARK-33165][SQL][TEST] Remove dependencies(scalatest,scalactic) from Benchmark
### What changes were proposed in this pull request?

This PR proposes to remove `assert` from `Benchmark` for making it easier to run benchmark codes via `spark-submit`.

### Why are the changes needed?

Since the current `Benchmark` (`master` and `branch-3.0`) has `assert`, we need to pass the proper jars of `scalatest` and `scalactic`;
 - scalatest-core_2.12-3.2.0.jar
 - scalatest-compatible-3.2.0.jar
 - scalactic_2.12-3.0.jar
```
./bin/spark-submit --jars scalatest-core_2.12-3.2.0.jar,scalatest-compatible-3.2.0.jar,scalactic_2.12-3.0.jar,./sql/catalyst/target/spark-catalyst_2.12-3.1.0-SNAPSHOT-tests.jar,./core/target/spark-core_2.12-3.1.0-SNAPSHOT-tests.jar --class org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark ./sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar --data-location /tmp/tpcds-sf1
```

This update can make developers submit benchmark codes without these dependencies;
```
./bin/spark-submit --jars ./sql/catalyst/target/spark-catalyst_2.12-3.1.0-SNAPSHOT-tests.jar,./core/target/spark-core_2.12-3.1.0-SNAPSHOT-tests.jar --class org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark ./sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar --data-location /tmp/tpcds-sf1
```

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

No.

### How was this patch tested?

Manually checked.

Closes #30064 from maropu/RemoveDepInBenchmark.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-16 11:39:09 +09:00
Denis Pyshev ba69d68d91 [SPARK-33080][BUILD] Replace fatal warnings snippet
### What changes were proposed in this pull request?

Current solution in build file to enable build failure on compilation warnings with exclusion of deprecation ones is not portable after SBT version 1.3.13 (build import fails with compilation error with SBT 1.4) and could be replaced with more robust and maintainable, especially since Scala 2.13.2 with similar built-in functionality.

Additionally, warnings were fixed to pass the build, with as few changes as possible:
warnings in 2.12 compilation fixed in code,
warnings in 2.13 compilation covered by configuration to be addressed separately

### Why are the changes needed?

Unblocks upgrade to SBT after 1.3.13.
Enhances build file maintainability.
Allows fine tune of warnings configuration in scope of Scala 2.13 compilation.

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

No.

### How was this patch tested?

`build/sbt`'s `compile` and `Test/compile` for both Scala 2.12 and 2.13 profiles.

Closes #29995 from gemelen/feature/warnings-reporter.

Authored-by: Denis Pyshev <git@gemelen.net>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-10-15 14:49:43 -05:00
Min Shen 82eea13c76 [SPARK-32915][CORE] Network-layer and shuffle RPC layer changes to support push shuffle blocks
### What changes were proposed in this pull request?

This is the first patch for SPIP SPARK-30602 for push-based shuffle.
Summary of changes:
* Introduce new API in ExternalBlockStoreClient to push blocks to a remote shuffle service.
* Leveraging the streaming upload functionality in SPARK-6237, it also enables the ExternalBlockHandler to delegate the handling of block push requests to MergedShuffleFileManager.
* Propose the API for MergedShuffleFileManager, where the core logic on the shuffle service side to handle block push requests is defined. The actual implementation of this API is deferred into a later RB to restrict the size of this PR.
* Introduce OneForOneBlockPusher to enable pushing blocks to remote shuffle services in shuffle RPC layer.
* New protocols in shuffle RPC layer to support the functionalities.

### Why are the changes needed?

Refer to the SPIP in SPARK-30602

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

### How was this patch tested?
Added unit tests.
The reference PR with the consolidated changes covering the complete implementation is also provided in SPARK-30602.
We have already verified the functionality and the improved performance as documented in the SPIP doc.

Lead-authored-by: Min Shen <mshenlinkedin.com>
Co-authored-by: Chandni Singh <chsinghlinkedin.com>
Co-authored-by: Ye Zhou <yezhoulinkedin.com>

Closes #29855 from Victsm/SPARK-32915.

Lead-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Ye Zhou <yezhou@linkedin.com>
Co-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Min Shen <victor.nju@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-10-15 12:34:52 -05:00
Dongjoon Hyun ec34a001ad [SPARK-33153][SQL][TESTS] Ignore Spark 2.4 in HiveExternalCatalogVersionsSuite on Python 3.8/3.9
### What changes were proposed in this pull request?

This PR aims to ignore Apache Spark 2.4.x distribution in HiveExternalCatalogVersionsSuite if Python version is 3.8 or 3.9.

### Why are the changes needed?

Currently, `HiveExternalCatalogVersionsSuite` is broken on the latest OS like `Ubuntu 20.04` because its default Python version is 3.8. PySpark 2.4.x doesn't work on Python 3.8 due to SPARK-29536.

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

No.

### How was this patch tested?

Manually.
```
$ python3 --version
Python 3.8.5

$ build/sbt "hive/testOnly *.HiveExternalCatalogVersionsSuite"
...
[info] All tests passed.
[info] Passed: Total 1, Failed 0, Errors 0, Passed 1
```

Closes #30044 from dongjoon-hyun/SPARK-33153.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-14 20:48:13 -07:00
Adam Binford 9ab0ec4e38 [SPARK-33146][CORE] Check for non-fatal errors when loading new applications in SHS
### What changes were proposed in this pull request?

Adds an additional check for non-fatal errors when attempting to add a new entry to the history server application listing.

### Why are the changes needed?

A bad rolling event log folder (missing appstatus file or no log files) would cause no applications to be loaded by the Spark history server. Figuring out why invalid event log folders are created in the first place will be addressed in separate issues, this just lets the history server skip the invalid folder and successfully load all the valid applications.

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

No

### How was this patch tested?

New UT

Closes #30037 from Kimahriman/bug/rolling-log-crashing-history.

Authored-by: Adam Binford <adam.binford@radiantsolutions.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-10-15 11:59:29 +09:00
neko 1bfcb51eeb [SPARK-33132][WEBUI] Make formatBytes return 0.0 B for negative input instead of NaN
### What changes were proposed in this pull request?
when bytesRead metric was negative, `formatBytes` in `ui.js` should just return `0.0 B` to avoid `NaN Undefined` result.

### Why are the changes needed?
Strengthen the parameter validataion to improve metric display on Summary Metrics of Spark  Stage UI.

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

### How was this patch tested?
It's a small change, just manual test.

Closes #30030 from akiyamaneko/formatBytes_NaN.

Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-13 09:29:05 -07:00
Holden Karau 7696ca5673 [SPARK-32881][CORE] Catch some race condition errors and log them more clearly
### What changes were proposed in this pull request?

Decommissioning can run out of time resulting in some race condition, these race conditions result in confusing error messages but not negative impact.

### Why are the changes needed?

The NPE & element missing errors in the log can create a missunderstanding.

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

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

Closes #29992 from holdenk/SPARK-32881-error-messaging-on-decom-race-messages.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-10 16:24:50 -07:00
yi.wu 0b326d5327 [SPARK-32857][CORE] Fix flaky o.a.s.s.BarrierTaskContextSuite.throw exception if the number of barrier() calls are not the same on every task
### What changes were proposed in this pull request?

Fix the flaky test.

### Why are the changes needed?

The test is flaky: `Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown`.

Check the full error stack [here](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128548/testReport/org.apache.spark.scheduler/BarrierTaskContextSuite/throw_exception_if_the_number_of_barrier___calls_are_not_the_same_on_every_task/).

By analyzing the log below, I found that task 0 hadn't reached the second `context.barrier()` when another three tasks already raised the sync timeout exceptions by the first `context.barrier()`. The timeout exceptions were caught by the `try...catch...`. Then, each task started another round barrier sync from the second `context.barrier()` and completed the sync successfully.

```scala
20/09/10 20:54:48.821 dispatcher-event-loop-10 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:48.822 dispatcher-event-loop-10 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 2, current progress: 1/4.
20/09/10 20:54:48.826 dispatcher-BlockManagerMaster INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on localhost:38420 (size: 2.2 KiB, free: 546.3 MiB)
20/09/10 20:54:48.908 dispatcher-event-loop-12 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:48.909 dispatcher-event-loop-12 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 1, current progress: 2/4.
20/09/10 20:54:48.959 dispatcher-event-loop-11 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:48.960 dispatcher-event-loop-11 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 3, current progress: 3/4.
20/09/10 20:54:49.616 dispatcher-CoarseGrainedScheduler INFO TaskSchedulerImpl: Skip current round of resource offers for barrier stage 0 because the barrier taskSet requires 4 slots, while the total number of available slots is 0.
20/09/10 20:54:49.899 dispatcher-event-loop-15 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:49.900 dispatcher-event-loop-15 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 1, current progress: 1/4.
20/09/10 20:54:49.965 dispatcher-event-loop-13 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:49.966 dispatcher-event-loop-13 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 3, current progress: 2/4.
20/09/10 20:54:50.112 dispatcher-event-loop-16 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:50.113 dispatcher-event-loop-16 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 0, current progress: 3/4.
20/09/10 20:54:50.609 dispatcher-CoarseGrainedScheduler INFO TaskSchedulerImpl: Skip current round of resource offers for barrier stage 0 because the barrier taskSet requires 4 slots, while the total number of available slots is 0.
20/09/10 20:54:50.826 dispatcher-event-loop-17 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0.
20/09/10 20:54:50.827 dispatcher-event-loop-17 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 2, current progress: 4/4.
20/09/10 20:54:50.827 dispatcher-event-loop-17 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received all updates from tasks, finished successfully.
```

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

No.

### How was this patch tested?

Updated the test and tested a hundred times without failure(Previously, there could be several failures).

Closes #29732 from Ngone51/fix-flaky-throw-exception.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-06 14:18:37 -07:00
Michael Munday b5e4b8c73e [SPARK-27428][CORE][TEST] Increase receive buffer size used in StatsdSinkSuite
### What changes were proposed in this pull request?

Increase size of socket receive buffer in these tests.

### Why are the changes needed?

The socket receive buffer size set in this test was too small for
the StatsdSinkSuite tests to run reliably on some systems. For a
test in this suite to run reliably the buffer needs to be large
enough to hold all the data in the packets being sent in a test
along with any additional kernel or protocol overhead. The amount
of kernel overhead per packet can vary from system to system but is
typically far higher than the protocol overhead.

If the receive buffer is too small and fills up then packets are
silently dropped. This leads to the test failing with a timeout.

If the socket defaults to a larger receive buffer (normally true)
then we should keep that size.

As well as increasing the minimum buffer size I've also decoupled
the datagram packet buffer size from the receive buffer size. The
receive buffer should in general be far larger to account for the
fact that multiple packets might be buffered, as well as the
aforementioned overhead. Any truncated data in individual packets
will be picked up by the tests.

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

No, this only affects the tests.

### How was this patch tested?
Existing tests on IBM Z and x86.

Closes #29819 from mundaym/fix-statsd.

Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-10-06 08:31:06 -05:00
Holden Karau db420f79cc [SPARK-33049][CORE] Decommission shuffle block test is flaky
### What changes were proposed in this pull request?

Increase the listener bus event length, syncrhonize the addition of blocks modified to the array list.

### Why are the changes needed?

This test appears flaky in Jenkins (can not repro locally). Given that the index file made it through and the index file is only transferred after the data file, the only two reasons I could come up with an interminentent failure here are with the listenerbus dropping a message or the two block change messages being received at the same time.

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

No (test only).

### How was this patch tested?

The tests still pass on my machine but they did before. We'll need to run it through jenkins a few times first.

Closes #29929 from holdenk/fix-.BlockManagerDecommissionIntegrationSuite.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-10-03 15:14:48 -07:00
Bo Yang 1299c8a81d [SPARK-33037][SHUFFLE] Remove knownManagers to support user's custom shuffle manager plugin
### What changes were proposed in this pull request?

Spark has a hardcode list to contain known shuffle managers, which has two values now. It does not contain user's custom shuffle manager which is set through Spark config "spark.shuffle.manager".

We hit issue when set "spark.shuffle.manager" with our own shuffle manager plugin (Uber Remote Shuffle Service implementation, https://github.com/uber/RemoteShuffleService). Other users will hit same issue when they implement their own shuffle manager.

It is better to remove that knownManagers hardcode list, to support user's custom shuffle manager implementation.

### Why are the changes needed?

Spark has shuffle manager API to support custom shuffle manager implementation. The hardcoded known managers list does not consider that shuffle manager config value which could be set by user. Thus better to remove this hardcoded known managers list.

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

### How was this patch tested?
Current Spark unit test already covers the code path.

Closes #29916 from boy-uber/knownManagers.

Lead-authored-by: Bo Yang <boy@uber.com>
Co-authored-by: Bo Yang <boy-uber@users.noreply.github.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-10-02 20:26:46 -07:00
Gabor Somogyi 991f7e81d4 [SPARK-32001][SQL] Create JDBC authentication provider developer API
### What changes were proposed in this pull request?
At the moment only the baked in JDBC connection providers can be used but there is a need to support additional databases and use-cases. In this PR I'm proposing a new developer API name `JdbcConnectionProvider`. To show how an external JDBC connection provider can be implemented I've created an example [here](https://github.com/gaborgsomogyi/spark-jdbc-connection-provider).

The PR contains the following changes:
* Added connection provider developer API
* Made JDBC connection providers constructor to noarg => needed to load them w/ service loader
* Connection providers are now loaded w/ service loader
* Added tests to load providers independently
* Moved `SecurityConfigurationLock` into a central place because other areas will change global JVM security config

### Why are the changes needed?
No custom authentication possibility.

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

### How was this patch tested?
* Existing + additional unit tests
* Docker integration tests
* Tested manually the newly created external JDBC connection provider

Closes #29024 from gaborgsomogyi/SPARK-32001.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-02 13:04:40 +09:00
Shruti Gumma 8657742ec7 [SPARK-32996][WEB-UI][FOLLOWUP] Move ExecutorSummarySuite to proper path
### What changes were proposed in this pull request?

This  change updates the test file location in #29872 to proper path.

### Why are the changes needed?

ExecutorSummarySuite.scala should be in core/src/test/scala instead of core/src/test/java.

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

### How was this patch tested?
Unit tests

Closes #29926 from shrutig/SPARK-32996.

Authored-by: Shruti Gumma <shruti_gumma@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-10-01 16:33:19 -07:00
Peter Toth 28ed3a512a [SPARK-32723][WEBUI] Upgrade to jQuery 3.5.1
### What changes were proposed in this pull request?
Upgrade to the latest available version of jQuery (3.5.1).

### Why are the changes needed?
There are some CVE-s reported (CVE-2020-11022, CVE-2020-11023) affecting older versions of jQuery. Although Spark UI is read-only and those CVEs doesn't seem to affect Spark, using the latest version of this library can help to handle vulnerability reports of security scans.

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

### How was this patch tested?
Manual tests and checked the jQuery 3.5 upgrade guide.

Closes #29902 from peter-toth/SPARK-32723-upgrade-to-jquery-3.5.1.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-30 21:30:17 -07:00
angerszhu 0b5a379c1f [SPARK-33023][CORE] Judge path of Windows need add condition Utils.isWindows
### What changes were proposed in this pull request?
according to  https://github.com/apache/spark/pull/29881#discussion_r496648397
we need add condition `Utils.isWindows`

### Why are the changes needed?
add strict condition of judging path is window path

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

### How was this patch tested?
No

Closes #29909 from AngersZhuuuu/SPARK-33023.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-30 19:24:50 -07:00
Dongjoon Hyun cc06266ade [SPARK-33019][CORE] Use spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=1 by default
### What changes were proposed in this pull request?

Apache Spark 3.1's default Hadoop profile is `hadoop-3.2`. Instead of having a warning documentation, this PR aims to use a consistent and safer version of Apache Hadoop file output committer algorithm which is `v1`. This will prevent a silent correctness regression during migration from Apache Spark 2.4/3.0 to Apache Spark 3.1.0. Of course, if there is a user-provided configuration, `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=2`, that will be used still.

### Why are the changes needed?

Apache Spark provides multiple distributions with Hadoop 2.7 and Hadoop 3.2. `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version` depends on the Hadoop version. Apache Hadoop 3.0 switches the default algorithm from `v1` to `v2` and now there exists a discussion to remove `v2`. We had better provide a consistent default behavior of `v1` across various Spark distributions.

- [MAPREDUCE-7282](https://issues.apache.org/jira/browse/MAPREDUCE-7282) MR v2 commit algorithm should be deprecated and not the default

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

Yes. This changes the default behavior. Users can override this conf.

### How was this patch tested?

Manual.

**BEFORE (spark-3.0.1-bin-hadoop3.2)**
```scala
scala> sc.version
res0: String = 3.0.1

scala> sc.hadoopConfiguration.get("mapreduce.fileoutputcommitter.algorithm.version")
res1: String = 2
```

**AFTER**
```scala
scala> sc.hadoopConfiguration.get("mapreduce.fileoutputcommitter.algorithm.version")
res0: String = 1
```

Closes #29895 from dongjoon-hyun/SPARK-DEFAUT-COMMITTER.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-29 12:02:45 -07:00
Akshat Bordia 7766fd13c9 [MINOR][DOCS] Fixing log message for better clarity
Fixing log message for better clarity.

Closes #29870 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-09-29 08:38:43 -05:00
Tom van Bussel f167002522 [SPARK-32901][CORE] Do not allocate memory while spilling UnsafeExternalSorter
### What changes were proposed in this pull request?

This PR changes `UnsafeExternalSorter` to no longer allocate any memory while spilling. In particular it removes the allocation of a new pointer array in `UnsafeInMemorySorter`. Instead the new pointer array is allocated whenever the next record is inserted into the sorter.

### Why are the changes needed?

Without this change the `UnsafeExternalSorter` could throw an OOM while spilling. The following sequence of events would have triggered an OOM:

1. `UnsafeExternalSorter` runs out of space in its pointer array and attempts to allocate a new large array to replace the old one.
2. `TaskMemoryManager` tries to allocate the memory backing the new large array using `MemoryManager`, but `MemoryManager` is only willing to return most but not all of the memory requested.
3. `TaskMemoryManager` asks `UnsafeExternalSorter` to spill, which causes `UnsafeExternalSorter` to spill the current run to disk, to free its record pages and to reset its `UnsafeInMemorySorter`.
4. `UnsafeInMemorySorter` frees the old pointer array, and tries to allocate a new small pointer array.
5. `TaskMemoryManager` tries to allocate the memory backing the small array using `MemoryManager`, but `MemoryManager` is unwilling to give it any memory, as the `TaskMemoryManager` is still holding on to the memory it got for the new large array.
6. `TaskMemoryManager` again asks `UnsafeExternalSorter` to spill, but this time there is nothing to spill.
7. `UnsafeInMemorySorter` receives less memory than it requested, and causes a `SparkOutOfMemoryError` to be thrown, which causes the current task to fail.

With the changes in the PR the following will happen instead:

1. `UnsafeExternalSorter` runs out of space in its pointer array and attempts to allocate a new large array to replace the old one.
2. `TaskMemoryManager` tries to allocate the memory backing the new large array using `MemoryManager`, but `MemoryManager` is only willing to return most but not all of the memory requested.
3. `TaskMemoryManager` asks `UnsafeExternalSorter` to spill, which causes `UnsafeExternalSorter` to spill the current run to disk, to free its record pages and to reset its `UnsafeInMemorySorter`.
4. `UnsafeInMemorySorter` frees the old pointer array.
5. `TaskMemoryManager` returns control to `UnsafeExternalSorter.growPointerArrayIfNecessary` (either by returning the the new large array or by throwing a `SparkOutOfMemoryError`).
6. `UnsafeExternalSorter` either frees the new large array or it ignores the `SparkOutOfMemoryError` depending on what happened in the previous step.
7. `UnsafeExternalSorter` successfully allocates a new small pointer array and operation continues as normal.

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

No

### How was this patch tested?

Tests were added in `UnsafeExternalSorterSuite` and `UnsafeInMemorySorterSuite`.

Closes #29785 from tomvanbussel/SPARK-32901.

Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-09-29 13:05:33 +02:00
Shruti Gumma 173da5bf11 [SPARK-32996][WEB-UI] Handle empty ExecutorMetrics in ExecutorMetricsJsonSerializer
### What changes were proposed in this pull request?
When `peakMemoryMetrics` in `ExecutorSummary` is `Option.empty`, then the `ExecutorMetricsJsonSerializer#serialize` method does not execute the `jsonGenerator.writeObject` method. This causes the json to be generated with `peakMemoryMetrics` key added to the serialized string, but no corresponding value.
This causes an error to be thrown when it is the next key `attributes` turn to be added to the json:
`com.fasterxml.jackson.core.JsonGenerationException: Can not write a field name, expecting a value
`

### Why are the changes needed?
At the start of the Spark job, if `peakMemoryMetrics` is `Option.empty`, then it causes
a `com.fasterxml.jackson.core.JsonGenerationException` to be thrown when we navigate to the Executors tab in Spark UI.
Complete stacktrace:

> com.fasterxml.jackson.core.JsonGenerationException: Can not write a field name, expecting a value
> 	at com.fasterxml.jackson.core.JsonGenerator._reportError(JsonGenerator.java:2080)
> 	at com.fasterxml.jackson.core.json.WriterBasedJsonGenerator.writeFieldName(WriterBasedJsonGenerator.java:161)
> 	at com.fasterxml.jackson.databind.ser.BeanPropertyWriter.serializeAsField(BeanPropertyWriter.java:725)
> 	at com.fasterxml.jackson.databind.ser.std.BeanSerializerBase.serializeFields(BeanSerializerBase.java:721)
> 	at com.fasterxml.jackson.databind.ser.BeanSerializer.serialize(BeanSerializer.java:166)
> 	at com.fasterxml.jackson.databind.ser.std.CollectionSerializer.serializeContents(CollectionSerializer.java:145)
> 	at com.fasterxml.jackson.module.scala.ser.IterableSerializer.serializeContents(IterableSerializerModule.scala:26)
> 	at com.fasterxml.jackson.module.scala.ser.IterableSerializer.serializeContents$(IterableSerializerModule.scala:25)
> 	at com.fasterxml.jackson.module.scala.ser.UnresolvedIterableSerializer.serializeContents(IterableSerializerModule.scala:54)
> 	at com.fasterxml.jackson.module.scala.ser.UnresolvedIterableSerializer.serializeContents(IterableSerializerModule.scala:54)
> 	at com.fasterxml.jackson.databind.ser.std.AsArraySerializerBase.serialize(AsArraySerializerBase.java:250)
> 	at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider._serialize(DefaultSerializerProvider.java:480)
> 	at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider.serializeValue(DefaultSerializerProvider.java:319)
> 	at com.fasterxml.jackson.databind.ObjectMapper._configAndWriteValue(ObjectMapper.java:4094)
> 	at com.fasterxml.jackson.databind.ObjectMapper.writeValueAsString(ObjectMapper.java:3404)
> 	at org.apache.spark.ui.exec.ExecutorsPage.allExecutorsDataScript$1(ExecutorsTab.scala:64)
> 	at org.apache.spark.ui.exec.ExecutorsPage.render(ExecutorsTab.scala:76)
> 	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.sparkproject.jetty.servlet.ServletHolder.handle(ServletHolder.java:873)
> 	at org.sparkproject.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1623)
> 	at org.apache.spark.ui.HttpSecurityFilter.doFilter(HttpSecurityFilter.scala:95)
> 	at org.sparkproject.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1610)
> 	at org.sparkproject.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:540)
> 	at org.sparkproject.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:255)
> 	at org.sparkproject.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1345)
> 	at org.sparkproject.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
> 	at org.sparkproject.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
> 	at org.sparkproject.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
> 	at org.sparkproject.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
> 	at org.sparkproject.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
> 	at org.sparkproject.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:753)
> 	at org.sparkproject.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:220)
> 	at org.sparkproject.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
> 	at org.sparkproject.jetty.server.Server.handle(Server.java:505)
> 	at org.sparkproject.jetty.server.HttpChannel.handle(HttpChannel.java:370)
> 	at org.sparkproject.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
> 	at org.sparkproject.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
> 	at org.sparkproject.jetty.io.FillInterest.fillable(FillInterest.java:103)
> 	at org.sparkproject.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
> 	at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.runTask(EatWhatYouKill.java:333)
> 	at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:310)
> 	at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:168)
> 	at org.sparkproject.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:126)
> 	at org.sparkproject.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:366)
> 	at org.sparkproject.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:698)
> 	at org.sparkproject.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:804)
> 	at java.base/java.lang.Thread.run(Thread.java:834)

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

### How was this patch tested?
Unit test

Closes #29872 from shrutig/SPARK-32996.

Authored-by: Shruti Gumma <shruti_gumma@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-09-28 10:07:36 -07:00
Chao Sun 8ccfbc114e [SPARK-32381][CORE][SQL] Move and refactor parallel listing & non-location sensitive listing to core
<!--
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 moves and refactors the parallel listing utilities from `InMemoryFileIndex` to Spark core so it can be reused by modules beside SQL. Along the process this also did some cleanups/refactorings:

- Created a `HadoopFSUtils` class under core
- Moved `InMemoryFileIndex.bulkListLeafFiles` into `HadoopFSUtils.parallelListLeafFiles`. It now depends on a `SparkContext` instead of `SparkSession` in SQL. Also added a few parameters which used to be read from `SparkSession.conf`: `ignoreMissingFiles`, `ignoreLocality`, `parallelismThreshold`, `parallelismMax ` and `filterFun` (for additional filtering support but we may be able to merge this with `filter` parameter in future).
- Moved `InMemoryFileIndex.listLeafFiles` into `HadoopFSUtils.listLeafFiles` with similar changes above.

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

Currently the locality-aware parallel listing mechanism only applies to `InMemoryFileIndex`. By moving this to core, we can potentially reuse the same mechanism for other code paths as well.

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

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

Since this is mostly a refactoring, it relies on existing unit tests such as those for `InMemoryFileIndex`.

Closes #29471 from sunchao/SPARK-32381.

Lead-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Chao Sun <sunchao@uber.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-09-24 10:58:52 -07:00
Holden Karau 27f6b5a103 [SPARK-32937][SPARK-32980][K8S] Fix decom & launcher tests and add some comments to reduce chance of breakage
### What changes were proposed in this pull request?

Fixes the log strings the decom integration tests looks for and add comments reminding people to run the K8s integration tests when changing those code paths.

### Why are the changes needed?

The strings it looks for have been changed.

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

No

### How was this patch tested?

WIP: Verify that the K8s jenkins job succeeds

Closes #29854 from holdenk/SPARK-32979-spark-k8s-decom-test-is-broken.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-23 15:39:31 -07:00
Zhen Li d01594e8d1 [SPARK-32886][WEBUI] fix 'undefined' link in event timeline view
### What changes were proposed in this pull request?

Fix ".../jobs/undefined" link from "Event Timeline" in jobs page. Job page link in "Event Timeline" view is constructed by fetching job page link defined in job list below. when job count exceeds page size of job table, only links of jobs in job table can be fetched from page. Other jobs' link would be 'undefined', and links of them in "Event Timeline" are broken, they are redirected to some wired URL like ".../jobs/undefined". This PR is fixing this wrong link issue. With this PR, job link in "Event Timeline" view would always redirect to correct job page.

### Why are the changes needed?

Wrong link (".../jobs/undefined") in "Event Timeline" of jobs page. for example, the first job in below page is not in table below, as job count(116) exceeds page size(100). When clicking it's item in "Event Timeline", page is redirected to ".../jobs/undefined", which is wrong. Links in "Event Timeline" should always be correct.
![undefinedlink](https://user-images.githubusercontent.com/10524738/93184779-83fa6d80-f6f1-11ea-8a80-1a304ca9cbb2.JPG)

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

No

### How was this patch tested?

Manually tested.

Closes #29757 from zhli1142015/fix-link-event-timeline-view.

Authored-by: Zhen Li <zhli@microsoft.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-21 09:05:40 -05:00
Wenchen Fan 0c66813ad9 Revert "[SPARK-32850][CORE] Simplify the RPC message flow of decommission"
This reverts commit 56ae95053d.
2020-09-21 13:28:31 +08:00
yi.wu f1dc479d39 [SPARK-32898][CORE] Fix wrong executorRunTime when task killed before real start
### What changes were proposed in this pull request?

Only calculate the executorRunTime when taskStartTimeNs > 0. Otherwise, set executorRunTime to 0.

### Why are the changes needed?

bug fix.

It's possible that a task be killed (e.g., by another successful attempt) before it reaches "taskStartTimeNs = System.nanoTime()". In this case, taskStartTimeNs is still 0 since it hasn't been really initialized. And we will get the wrong executorRunTime by calculating System.nanoTime() - taskStartTimeNs.

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

Yes, users will see the correct executorRunTime.

### How was this patch tested?

Pass existing tests.

Closes #29789 from Ngone51/fix-SPARK-32898.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-09-18 14:02:14 -07:00
Tom van Bussel 105225ddbc [SPARK-32911][CORE] Free memory in UnsafeExternalSorter.SpillableIterator.spill() when all records have been read
### What changes were proposed in this pull request?

This PR changes `UnsafeExternalSorter.SpillableIterator` to free its memory (except for the page holding the last record) if it is forced to spill after all of its records have been read. It also makes sure that `lastPage` is freed if `loadNext` is never called the again. The latter was necessary to get my test case to succeed (otherwise it would complain about a leak).

### Why are the changes needed?

No memory is freed after calling `UnsafeExternalSorter.SpillableIterator.spill()` when all records have been read, even though it is still holding onto some memory. This may cause a `SparkOutOfMemoryError` to be thrown, even though we could have just freed the memory instead.

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

No

### How was this patch tested?

A test was added to `UnsafeExternalSorterSuite`.

Closes #29787 from tomvanbussel/SPARK-32911.

Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-18 11:49:26 +00:00
William Hyun 7892887981 [SPARK-32930][CORE] Replace deprecated isFile/isDirectory methods
### What changes were proposed in this pull request?

This PR aims to replace deprecated `isFile` and `isDirectory` methods.

```diff
- fs.isDirectory(hadoopPath)
+ fs.getFileStatus(hadoopPath).isDirectory
```

```diff
- fs.isFile(new Path(inProgressLog))
+ fs.getFileStatus(new Path(inProgressLog)).isFile
```

### Why are the changes needed?

It shows deprecation warnings.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-3.2-hive-2.3/1244/consoleFull

```
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-3.2-hive-2.3/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala:815: method isFile in class FileSystem is deprecated: see corresponding Javadoc for more information.
[warn]             if (!fs.isFile(new Path(inProgressLog))) {
```

```
[warn] /home/jenkins/workspace/spark-master-test-sbt-hadoop-3.2-hive-2.3/core/src/main/scala/org/apache/spark/SparkContext.scala:1884: method isDirectory in class FileSystem is deprecated: see corresponding Javadoc for more information.
[warn]           if (fs.isDirectory(hadoopPath)) {
```

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

No.

### How was this patch tested?

Pass the Jenkins.

Closes #29796 from williamhyun/filesystem.

Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-18 18:13:11 +09:00
yi.wu a54a6a0113 [SPARK-32287][CORE] Fix flaky o.a.s.ExecutorAllocationManagerSuite on GithubActions
### What changes were proposed in this pull request?

To fix the flaky `ExecutorAllocationManagerSuite`: Avoid first `schedule()` invocation after `ExecutorAllocationManager` started.

### Why are the changes needed?

`ExecutorAllocationManagerSuite` is still flaky, see:

https://github.com/apache/spark/pull/29722/checks?check_run_id=1117979237

By checking the below logs, we can see that there's a race condition between thread `pool-1-thread-1-ScalaTest-running` and thread `spark-dynamic-executor-allocation`.  The only possibility of thread `spark-dynamic-executor-allocation` becoming active is the first time invocation of `schedule()`(since the `TEST_SCHEDULE_INTERVAL`(30s) is really long, so it's impossible the second invocation would happen).  Thus, I think we shall avoid the first invocation too.

```scala
20/09/15 12:41:20.831 pool-1-thread-1-ScalaTest-running-ExecutorAllocationManagerSuite INFO ExecutorAllocationManager: Requesting 1 new executor because tasks are backlogged (new desired total will be 2 for resource profile id: 0)
20/09/15 12:41:20.832 spark-dynamic-executor-allocation INFO ExecutorAllocationManager: Requesting 2 new executors because tasks are backlogged (new desired total will be 4 for resource profile id: 0)
```

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

No.

### How was this patch tested?

The flaky can't be reproduced locally so it's hard to say it has been completely fixed by now. We need time to see the result.

Closes #29773 from Ngone51/fix-SPARK-32287.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-17 11:20:50 +00:00
Tom van Bussel e5e54a3614 [SPARK-32900][CORE] Allow UnsafeExternalSorter to spill when there are nulls
### What changes were proposed in this pull request?

This PR changes the way `UnsafeExternalSorter.SpillableIterator` checks whether it has spilled already, by checking whether `inMemSorter` is null. It also allows it to spill other `UnsafeSorterIterator`s than `UnsafeInMemorySorter.SortedIterator`.

### Why are the changes needed?

Before this PR `UnsafeExternalSorter.SpillableIterator` could not spill when there are NULLs in the input and radix sorting is used. Currently, Spark determines whether UnsafeExternalSorter.SpillableIterator has not spilled yet by checking whether `upstream` is an instance of `UnsafeInMemorySorter.SortedIterator`. When radix sorting is used and there are NULLs in the input however, `upstream` will be an instance of `UnsafeExternalSorter.ChainedIterator` instead, and Spark will assume that the `SpillableIterator` iterator has spilled already, and therefore cannot spill again when it's supposed to spill.

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

No

### How was this patch tested?

A test was added to `UnsafeExternalSorterSuite` (and therefore also to `UnsafeExternalSorterRadixSortSuite`). I manually confirmed that the test failed in `UnsafeExternalSorterRadixSortSuite` without this patch.

Closes #29772 from tomvanbussel/SPARK-32900.

Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-09-17 12:35:40 +02:00
yi.wu 56ae95053d [SPARK-32850][CORE] Simplify the RPC message flow of decommission
### What changes were proposed in this pull request?

This PR cleans up the RPC message flow among the multiple decommission use cases, it includes changes:

* Keep `Worker`'s decommission status be consistent between the case where decommission starts from `Worker` and the case where decommission starts from the `MasterWebUI`: sending `DecommissionWorker` from `Master` to `Worker` in the latter case.

* Change from two-way communication to one-way communication when notifying decommission between driver and executor: it's obviously unnecessary for the executor to acknowledge the decommission status to the driver since the decommission request is from the driver. And it's same in reverse.

* Only send one message instead of two(`DecommissionSelf`/`DecommissionBlockManager`) when decommission the executor: executor and `BlockManager` are in the same JVM.

* Clean up codes around here.

### Why are the changes needed?

Before:

<img width="1948" alt="WeChat56c00cc34d9785a67a544dca036d49da" src="https://user-images.githubusercontent.com/16397174/92850308-dc461c80-f41e-11ea-8ac0-287825f4e0c4.png">

After:
<img width="1968" alt="WeChat05f7afb017e3f0132394c5e54245e49e" src="https://user-images.githubusercontent.com/16397174/93189571-de88dd80-f774-11ea-9300-1943920aa27d.png">

(Note the diagrams only counts those RPC calls that needed to go through the network. Local RPC calls are not counted here.)

After this change, We reduced 6 original RPC calls and added one more RPC call for keeping the consistent decommission status for the Worker. And the RPC flow becomes more clear.

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

No.

### How was this patch tested?

Updated existing tests.

Closes #29722 from Ngone51/simplify-decommission-rpc.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-16 15:00:31 +00:00
Zhenhua Wang 99384d1e83 [SPARK-32738][CORE] Should reduce the number of active threads if fatal error happens in Inbox.process
### What changes were proposed in this pull request?

Processing for `ThreadSafeRpcEndpoint` is controlled by  `numActiveThreads` in `Inbox`. Now if any fatal error happens during `Inbox.process`, `numActiveThreads` is not reduced. Then other threads can not process messages in that inbox, which causes the endpoint to "hang". For other type of endpoints, we also should keep  `numActiveThreads` correct.

This problem is more serious in previous Spark 2.x versions since the driver, executor and block manager endpoints are all thread safe endpoints.

To fix this, we should reduce the number of active threads if fatal error happens in `Inbox.process`.

### Why are the changes needed?

`numActiveThreads` is not correct when fatal error happens and will cause the described problem.

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

No.

### How was this patch tested?

Add a new test.

Closes #29580 from wzhfy/deal_with_fatal_error.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 06:46:17 +00:00
yi.wu 0811666ab1 [SPARK-32878][CORE] Avoid scheduling TaskSetManager which has no pending tasks
### What changes were proposed in this pull request?

This PR proposes to avoid scheduling the (non-zombie) TaskSetManager which has no pending tasks.

### Why are the changes needed?

Currently, Spark always tries to schedule a (non-zombie) TaskSetManager even if it has no pending tasks. This causes notable problems for the barrier TaskSetManager: 1. `calculateAvailableSlots` can be called for multiple times for a launched barrier TaskSetManager; 2. user would see "Skip current round of resource offers for barrier stage" log message for
a launched barrier TaskSetManager all the time until the barrier TaskSetManager finishes, which is quite confused.

Besides, scheduling a TaskSetManager always involves many function invocations even if there're no pending tasks.

Therefore, I think we can skip those un-schedulable TasksetManagers to avoid the potential overhead.

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

No.

### How was this patch tested?

Pass existing tests.

Closes #29750 from Ngone51/filter-out-unschedulable-stage.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 21:15:06 -07:00
LantaoJin 7a9b066c66 [SPARK-32715][CORE] Fix memory leak when failed to store pieces of broadcast
### What changes were proposed in this pull request?
In TorrentBroadcast.scala
```scala
L133: if (!blockManager.putSingle(broadcastId, value, MEMORY_AND_DISK, tellMaster = false))
L137: TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec)
L147: if (!blockManager.putBytes(pieceId, bytes, MEMORY_AND_DISK_SER, tellMaster = true))
```
After the original value is saved successfully(TorrentBroadcast.scala: L133), but the following `blockifyObject()`(L137) or store piece(L147) steps are failed. There is no opportunity to release broadcast from memory.

This patch is to remove all pieces of the broadcast when failed to blockify or failed to store some pieces of a broadcast.

### Why are the changes needed?
We use Spark thrift-server as a long-running service. A bad query submitted a heavy BroadcastNestLoopJoin operation and made driver full GC. We killed the bad query but we found the driver's memory usage was still high and full GCs were still frequent. By investigating with GC dump and log, we found the broadcast may memory leak.

> 2020-08-19T18:54:02.824-0700: [Full GC (Allocation Failure)
2020-08-19T18:54:02.824-0700: [Class Histogram (before full gc):
116G->112G(170G), 184.9121920 secs]
[Eden: 32.0M(7616.0M)->0.0B(8704.0M) Survivors: 1088.0M->0.0B Heap: 116.4G(170.0G)->112.9G(170.0G)], [Metaspace: 177285K->177270K(182272K)]
1: 676531691 72035438432 [B
2: 676502528 32472121344 org.apache.spark.sql.catalyst.expressions.UnsafeRow
3: 99551 12018117568 [Ljava.lang.Object;
4: 26570 4349629040 [I
5: 6 3264536688 [Lorg.apache.spark.sql.catalyst.InternalRow;
6: 1708819 256299456 [C
7: 2338 179615208 [J
8: 1703669 54517408 java.lang.String
9: 103860 34896960 org.apache.spark.status.TaskDataWrapper
10: 177396 25545024 java.net.URI
...

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

### How was this patch tested?
Manually test. This UT is hard to write and the patch is straightforward.

Closes #29558 from LantaoJin/SPARK-32715.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 18:24:52 -07:00
Ankur Dave 72550c3be7 [SPARK-32872][CORE] Prevent BytesToBytesMap at MAX_CAPACITY from exceeding growth threshold
### What changes were proposed in this pull request?

When BytesToBytesMap is at `MAX_CAPACITY` and reaches its growth threshold, `numKeys >= growthThreshold` is true but `longArray.size() / 2 < MAX_CAPACITY` is false. This correctly prevents the map from growing, but `canGrowArray` incorrectly remains true. Therefore the map keeps accepting new keys and exceeds its growth threshold. If we attempt to spill the map in this state, the UnsafeKVExternalSorter will not be able to reuse the long array for sorting. By this point the task has typically consumed all available memory, so the allocation of the new pointer array is likely to fail.

This PR fixes the issue by setting `canGrowArray` to false in this case. This prevents the map from accepting new elements when it cannot grow to accommodate them.

### Why are the changes needed?

Without this change, hash aggregations will fail when the number of groups per task is greater than `MAX_CAPACITY / 2 = 2^28` (approximately 268 million), and when the grouping aggregation is the only memory-consuming operator in its stage.

For example, the final aggregation in `SELECT COUNT(DISTINCT id) FROM tbl` fails when `tbl` contains 1 billion distinct values and when `spark.sql.shuffle.partitions=1`.

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

No.

### How was this patch tested?

Reproducing this issue requires building a very large BytesToBytesMap. Because this is infeasible to do in a unit test, this PR was tested manually by adding the following test to AbstractBytesToBytesMapSuite. Before this PR, the test fails in 8.5 minutes. With this PR, the test passes in 1.5 minutes.

```java
public abstract class AbstractBytesToBytesMapSuite {
  // ...
  Test
  public void respectGrowthThresholdAtMaxCapacity() {
    TestMemoryManager memoryManager2 =
        new TestMemoryManager(
            new SparkConf()
            .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true)
            .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 25600 * 1024 * 1024L)
            .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)
            .set(package$.MODULE$.SHUFFLE_COMPRESS(), false));
    TaskMemoryManager taskMemoryManager2 = new TaskMemoryManager(memoryManager2, 0);
    final long pageSizeBytes = 8000000 + 8; // 8 bytes for end-of-page marker
    final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager2, 1024, pageSizeBytes);

    try {
      // Insert keys into the map until it stops accepting new keys.
      for (long i = 0; i < BytesToBytesMap.MAX_CAPACITY; i++) {
        if (i % (1024 * 1024) == 0) System.out.println("Inserting element " + i);
        final long[] value = new long[]{i};
        BytesToBytesMap.Location loc = map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8);
        Assert.assertFalse(loc.isDefined());
        boolean success =
            loc.append(value, Platform.LONG_ARRAY_OFFSET, 8, value, Platform.LONG_ARRAY_OFFSET, 8);
        if (!success) break;
      }

      // The map should grow to its max capacity.
      long capacity = map.getArray().size() / 2;
      Assert.assertTrue(capacity == BytesToBytesMap.MAX_CAPACITY);

      // The map should stop accepting new keys once it has reached its growth
      // threshold, which is half the max capacity.
      Assert.assertTrue(map.numKeys() == BytesToBytesMap.MAX_CAPACITY / 2);

      map.free();
    } finally {
      map.free();
    }
  }
}
```

Closes #29744 from ankurdave/SPARK-32872.

Authored-by: Ankur Dave <ankurdave@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 13:58:15 -07:00
gengjiaan e558b8a0fd [SPARK-31847][CORE][TESTS] DAGSchedulerSuite: Rewrite the test framework to support apply specified spark configurations
### What changes were proposed in this pull request?
`DAGSchedulerSuite` exists some issue:
`afterEach` and `init` are called when the `SparkConf` of the default `SparkContext` has no configuration that the test case must set. This causes the `SparkContext` initialized in `beforeEach` to be discarded without being used, resulting in waste. On the other hand, the flexibility to add configurations to `SparkConf` should be addressed by the test framework.

Test suites inherits `LocalSparkContext` can be simplified.

### Why are the changes needed?
Reduce overhead about init `SparkContext`.
Rewrite the test framework to support apply specified spark configurations.

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

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

Closes #29228 from beliefer/extend-test-frame-for-dag.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-14 11:57:29 +09:00
yangjie01 513d51a2c5 [SPARK-32808][SQL] Fix some test cases of sql/core module in scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to partial resolve [SPARK-32808](https://issues.apache.org/jira/browse/SPARK-32808), total of 26 failed test cases were fixed, the related suite as follow:

- `StreamingAggregationSuite` related test cases (2 FAILED -> Pass)

- `GeneratorFunctionSuite` related test cases (2 FAILED -> Pass)

- `UDFSuite` related test cases (2 FAILED -> Pass)

- `SQLQueryTestSuite` related test cases (5 FAILED -> Pass)

- `WholeStageCodegenSuite` related test cases (1 FAILED -> Pass)

- `DataFrameSuite` related test cases (3 FAILED -> Pass)

- `OrcV1QuerySuite\OrcV2QuerySuite` related test cases (4 FAILED -> Pass)

- `ExpressionsSchemaSuite` related test cases (1 FAILED -> Pass)

- `DataFrameStatSuite` related test cases (1 FAILED -> Pass)

- `JsonV1Suite\JsonV2Suite\JsonLegacyTimeParserSuite` related test cases (6 FAILED -> Pass)

The main change of this pr as following:

- Fix Scala 2.13 compilation problems in   `ShuffleBlockFetcherIterator`  and `Analyzer`

- Specified `Seq` to `scala.collection.Seq` in `objects.scala` and `GenericArrayData` because internal use `Seq` maybe `mutable.ArraySeq` and not easy to call `.toSeq`

- Should specified `Seq` to `scala.collection.Seq`  when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but `Seq` is `immutable.Seq` in Scala 2.13

- Use a compatible way to let `+` and `-` method  of `Decimal` having the same behavior in Scala 2.12 and Scala 2.13

- Call `toList` in `RelationalGroupedDataset.toDF` method when `groupingExprs` is `Stream` type because `Stream` can't serialize in Scala 2.13

- Add a manual sort to `classFunsMap` in `ExpressionsSchemaSuite` because `Iterable.groupBy` in Scala 2.13 has different result with `TraversableLike.groupBy`  in Scala 2.12

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

Should specified `Seq` to `scala.collection.Seq`  when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but the `Seq` is `immutable.Seq` in Scala 2.13

### 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
mvn clean install -DskipTests  -pl sql/core -Pscala-2.13 -am
mvn test -pl sql/core -Pscala-2.13
```

**Before**
```
Tests: succeeded 8166, failed 319, canceled 1, ignored 52, pending 0
*** 319 TESTS FAILED ***

```

**After**

```
Tests: succeeded 8204, failed 286, canceled 1, ignored 52, pending 0
*** 286 TESTS FAILED ***

```

Closes #29660 from LuciferYang/SPARK-32808.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-09 08:53:44 -05:00
Thomas Graves 514bf563a7 [SPARK-32823][WEB UI] Fix the master ui resources reporting
### What changes were proposed in this pull request?

Fixes the master UI for properly summing the resources total across multiple workers.
field:
Resources in use: 0 / 8 gpu

The bug here is that it was creating MutableResourceInfo and then reducing using the + operator.  the + operator in MutableResourceInfo simple adds the address from one to the addresses of the other.  But its using a HashSet so if the addresses are the same then you lose the correct amount.  ie worker1 has gpu addresses 0,1,2,3 and worker2 has addresses 0,1,2,3 then you only see 4 total GPUs when there are 8.

In this case we don't really need to create the MutableResourceInfo at all because we just want the sums for used and total so just remove the use of it.  The other uses of it are per Worker so those should be ok.

### Why are the changes needed?

fix UI

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

UI

### How was this patch tested?

tested manually on standalone cluster with multiple workers and multiple GPUs and multiple fpgas

Closes #29683 from tgravescs/SPARK-32823.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-09 10:33:21 +09:00
Thomas Graves e8634d8f6f [SPARK-32824][CORE] Improve the error message when the user forgets the .amount in a resource config
### What changes were proposed in this pull request?

If the user forgets to specify .amount on a resource config like spark.executor.resource.gpu, the error message thrown is very confusing:

```
ERROR SparkContext: Error initializing SparkContext.java.lang.StringIndexOutOfBoundsException: String index out of range:
-1 at java.lang.String.substring(String.java:1967) at
```

This makes it so we have a readable error thrown

### Why are the changes needed?

confusing error for users
### Does this PR introduce _any_ user-facing change?

just error message

### How was this patch tested?

Tested manually on standalone cluster

Closes #29685 from tgravescs/SPARK-32824.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-09 10:28:40 +09:00