Commit graph

166 commits

Author SHA1 Message Date
Dongjoon Hyun f9f95686cb [SPARK-35996][BUILD] Setting version to 3.3.0-SNAPSHOT
### What changes were proposed in this pull request?

This PR aims to update `master` branch version to 3.3.0-SNAPSHOT.

### Why are the changes needed?

Start to prepare Apache Spark 3.3.0 and the published snapshot version should not conflict with `branch-3.2`.

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

N/A.

### How was this patch tested?

Pass the CIs.

Closes #33196 from dongjoon-hyun/SPARK-35996.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 13:47:36 -07:00
Chandni Singh 8ce1e344e5 [SPARK-35671][SHUFFLE][CORE] Add support in the ESS to serve merged shuffle block meta and data to executors
### What changes were proposed in this pull request?
This adds support in the ESS to serve merged shuffle block meta and data requests to executors.
This change is needed for fetching remote merged shuffle data from the remote shuffle services. This is part of push-based shuffle SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).

This change introduces new messages between clients and the external shuffle service:

1. `MergedBlockMetaRequest`: The client sends this to external shuffle to get the meta information for a merged block. The response to this is one of these :
  - `MergedBlockMetaSuccess` : contains request id, number of chunks, and a `ManagedBuffer` which is a `FileSegmentBuffer` backed by the merged block meta file.
  - `RpcFailure`: this is sent back to client in case of failure. This is an existing message.

2. `FetchShuffleBlockChunks`: This is similar to `FetchShuffleBlocks` message but it is to fetch merged shuffle chunks instead of blocks.

### Why are the changes needed?
These changes are needed for push-based shuffle. Refer to the SPIP in [SPARK-30602](https://issues.apache.org/jira/browse/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](https://issues.apache.org/jira/browse/SPARK-30602).
We have already verified the functionality and the improved performance as documented in the SPIP doc.

Lead-authored-by: Chandni Singh chsinghlinkedin.com
Co-authored-by: Min Shen mshenlinkedin.com

Closes #32811 from otterc/SPARK-35671.

Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-06-20 17:22:37 -05:00
yi.wu 00b63c8dc2 [SPARK-27991][CORE] Defer the fetch request on Netty OOM
### What changes were proposed in this pull request?

This PR proposes a workaround to address the Netty OOM issue (SPARK-24989, SPARK-27991):

Basically, `ShuffleBlockFetcherIterator` would catch the `OutOfDirectMemoryError` from Netty and then set a global flag for the shuffle module. Any pending fetch requests would be deferred if there're in-flight requests until the flag is unset. And the flag will be unset when there's a fetch request succeed.

Note that catching the Netty OOM rather than abort the application is feasible because Netty manage its own memory region (offheap by default) separately. So Netty OOM doesn't mean the memory shortage of Spark.

### Why are the changes needed?

The Netty OOM issue is a very corner case. It usually happens in the large-scale cluster, where a reduce task could fetch shuffle blocks from hundreds of nodes concurrently in a short time. Internally, we found a cluster that has created 260+ clients within 6s before throwing Netty OOM.

Although Spark has configurations, e.g., `spark.reducer.maxReqsInFlight` to tune the number of concurrent requests, it's usually not a easy decision for the user to set a reasonable value regarding the workloads, machine resources, etc. But with this fix, Spark would heal the Netty memory issue itself without any specific configurations.

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

No.

### How was this patch tested?

Added unit tests.

Closes #32287 from Ngone51/SPARK-27991.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-20 04:26:56 +00:00
yangjie01 c7e18ad223 [SPARK-35132][BUILD][CORE] Upgrade netty-all to 4.1.63.Final
### What changes were proposed in this pull request?
There are 3 CVE problems were found after netty 4.1.51.Final as follows:

- [CVE-2021-21409](https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2021-21409)
- [CVE-2021-21295](https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2021-21295)
- [CVE-2021-21290](https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2021-21290)

So the main change of this pr is upgrade netty-all to 4.1.63.Final avoid these potential risks.

Another change is to clean up deprecated api usage: [Tiny caches have been merged into small caches](https://github.com/netty/netty/blob/4.1/buffer/src/main/java/io/netty/buffer/PooledByteBufAllocator.java#L447-L455)(after [netty#10267](https://github.com/netty/netty/pull/10267)) and [should use  PooledByteBufAllocator(boolean, int, int, int, int, int, int, boolean, int)](https://github.com/netty/netty/blob/4.1/buffer/src/main/java/io/netty/buffer/PooledByteBufAllocator.java#L227-L239) api to create `PooledByteBufAllocator`.

### Why are the changes needed?
Upgrade netty-all to 4.1.63.Final avoid CVE problems.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #32227 from LuciferYang/SPARK-35132.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-04-20 18:28:43 -05:00
weixiuli bf9f3b884f [SPARK-34834][NETWORK] Fix a potential Netty memory leak in TransportResponseHandler
### What changes were proposed in this pull request?
There is a potential Netty memory leak in TransportResponseHandler.

### Why are the changes needed?
Fix a potential Netty memory leak in TransportResponseHandler.

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

### How was this patch tested?
NO

Closes #31942 from weixiuli/SPARK-34834.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-04-14 11:44:48 -05:00
HyukjinKwon a153efa643 [SPARK-35002][YARN][TESTS][FOLLOW-UP] Fix java.net.BindException in MiniYARNCluster
### What changes were proposed in this pull request?

This PR fixes two tests below:

https://github.com/apache/spark/runs/2320161984

```
[info] YarnShuffleIntegrationSuite:
[info] org.apache.spark.deploy.yarn.YarnShuffleIntegrationSuite *** ABORTED *** (228 milliseconds)
[info]   org.apache.hadoop.yarn.exceptions.YarnRuntimeException: org.apache.hadoop.yarn.webapp.WebAppException: Error starting http server
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster.startResourceManager(MiniYARNCluster.java:373)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster.access$300(MiniYARNCluster.java:128)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper.serviceStart(MiniYARNCluster.java:503)
[info]   at org.apache.hadoop.service.AbstractService.start(AbstractService.java:194)
[info]   at org.apache.hadoop.service.CompositeService.serviceStart(CompositeService.java:121)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster.serviceStart(MiniYARNCluster.java:322)
[info]   at org.apache.hadoop.service.AbstractService.start(AbstractService.java:194)
[info]   at org.apache.spark.deploy.yarn.BaseYarnClusterSuite.beforeAll(BaseYarnClusterSuite.scala:95)
...
[info]   Cause: java.net.BindException: Port in use: fv-az186-831:0
[info]   at org.apache.hadoop.http.HttpServer2.constructBindException(HttpServer2.java:1231)
[info]   at org.apache.hadoop.http.HttpServer2.bindForSinglePort(HttpServer2.java:1253)
[info]   at org.apache.hadoop.http.HttpServer2.openListeners(HttpServer2.java:1316)
[info]   at org.apache.hadoop.http.HttpServer2.start(HttpServer2.java:1167)
[info]   at org.apache.hadoop.yarn.webapp.WebApps$Builder.start(WebApps.java:449)
[info]   at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.startWepApp(ResourceManager.java:1247)
[info]   at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceStart(ResourceManager.java:1356)
[info]   at org.apache.hadoop.service.AbstractService.start(AbstractService.java:194)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster.startResourceManager(MiniYARNCluster.java:365)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster.access$300(MiniYARNCluster.java:128)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper.serviceStart(MiniYARNCluster.java:503)
[info]   at org.apache.hadoop.service.AbstractService.start(AbstractService.java:194)
[info]   at org.apache.hadoop.service.CompositeService.serviceStart(CompositeService.java:121)
[info]   at org.apache.hadoop.yarn.server.MiniYARNCluster.serviceStart(MiniYARNCluster.java:322)
[info]   at org.apache.hadoop.service.AbstractService.start(AbstractService.java:194)
[info]   at org.apache.spark.deploy.yarn.BaseYarnClusterSuite.beforeAll(BaseYarnClusterSuite.scala:95)
[info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212)
[info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
[info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
[info]   at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:61)
...
```

https://github.com/apache/spark/runs/2323342094

```
[info] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testBadSecret started
[error] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testBadSecret failed: java.lang.AssertionError: Connecting to /10.1.0.161:39895 timed out (120000 ms), took 120.081 sec
[error]     at org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testBadSecret(ExternalShuffleSecuritySuite.java:85)
[error]     ...
[info] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testBadAppId started
[error] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testBadAppId failed: java.lang.AssertionError: Connecting to /10.1.0.198:44633 timed out (120000 ms), took 120.08 sec
[error]     at org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testBadAppId(ExternalShuffleSecuritySuite.java:76)
[error]     ...
[info] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testValid started
[error] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testValid failed: java.io.IOException: Connecting to /10.1.0.119:43575 timed out (120000 ms), took 120.089 sec
[error]     at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:285)
[error]     at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:218)
[error]     at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:230)
[error]     at org.apache.spark.network.shuffle.ExternalBlockStoreClient.registerWithShuffleServer(ExternalBlockStoreClient.java:211)
[error]     at org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.validate(ExternalShuffleSecuritySuite.java:108)
[error]     at org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testValid(ExternalShuffleSecuritySuite.java:68)
[error]     ...
[info] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testEncryption started
[error] Test org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testEncryption failed: java.io.IOException: Connecting to /10.1.0.248:35271 timed out (120000 ms), took 120.014 sec
[error]     at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:285)
[error]     at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:218)
[error]     at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:230)
[error]     at org.apache.spark.network.shuffle.ExternalBlockStoreClient.registerWithShuffleServer(ExternalBlockStoreClient.java:211)
[error]     at org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.validate(ExternalShuffleSecuritySuite.java:108)
[error]     at org.apache.spark.network.shuffle.ExternalShuffleSecuritySuite.testEncryption(ExternalShu
```

For Yarn cluster suites, its difficult to fix. This PR makes it skipped if it fails to bind.
For shuffle related suites, it uses local host

### Why are the changes needed?

To make the tests stable

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

No, dev-only.

### How was this patch tested?

Its tested in GitHub Actions: https://github.com/HyukjinKwon/spark/runs/2340210765

Closes #32126 from HyukjinKwon/SPARK-35002-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-04-14 17:13:48 +08:00
Kent Yao 5692aa0c2c [SPARK-34894][CORE] Use 'io.connectionTimeout' as a hint instead of 'spark.network.timeout' for lost connections
### What changes were proposed in this pull request?

Currently, when a connection for TransportClient is marked as idled and closed, we suggest users adjust `spark.network.timeout` for all transport modules. As a lot of timeout configs will fallback to the `spark.network.timeout`, this could be a piece of overkill advice, we should give a more targeted one with `spark.${moduleName}.io.connectionTimeout`

### Why are the changes needed?

better advise for overloaded network traffic cases

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

yes, when a connection is zombied and closed by spark internally, users can use a more targeted config to tune their jobs
### How was this patch tested?

Just log and doc. Passing Jenkins and GA

Closes #31990 from yaooqinn/SPARK-34894.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-30 09:58:24 +08:00
yangjie01 e757091820 [SPARK-34722][CORE][SQL][TEST] Clean up deprecated API usage related to JUnit4
### What changes were proposed in this pull request?
The main change of this pr as follows:

- Use `org.junit.Assert.assertThrows(String, Class, ThrowingRunnable)` method instead of  `ExpectedException.none()`
- Use `org.hamcrest.MatcherAssert.assertThat()` method instead of   `org.junit.Assert.assertThat(T, org.hamcrest.Matcher<? super T>)`

### Why are the changes needed?
Clean up deprecated API usage

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31815 from LuciferYang/SPARK-34722.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-14 23:33:03 -07:00
Chandni Singh d00f0695b7 [SPARK-32917][SHUFFLE][CORE] Adds support for executors to push shuffle blocks after successful map task completion
### What changes were proposed in this pull request?
This is the shuffle writer side change where executors can push data to remote shuffle services. This is needed for push-based shuffle - SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
Summary of changes:
- This adds support for executors to push shuffle blocks after map tasks complete writing shuffle data.
- This also introduces a timeout specifically for creating connection to remote shuffle services.

### Why are the changes needed?
- These changes are needed for push-based shuffle. Refer to the SPIP in [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
- The main reason to create a separate connection creation timeout is because the existing `connectionTimeoutMs` is overloaded and is used for connection creation timeouts as well as connection idle timeout. The connection creation timeout should be much lower than the idle timeouts. The default for `connectionTimeoutMs` is 120s. This is quite high for just establishing the connections.  If a shuffle server node is bad then the connection creation will fail within few seconds. However, an overloaded shuffle server may take much longer to respond to a request and the channel can stay idle for a much longer time which is expected.  Another reason is that with push-based shuffle, an executor may be fetching shuffle data and pushing shuffle data (next stage) simultaneously. Both these tasks will share the same connections with the shuffle service. If there is a bad shuffle server node and the connection creation timeout is very high then both these tasks end up waiting a long time time eventually impacting the performance.

### Does this PR introduce _any_ user-facing change?
Yes. This PR introduces client-side configs for push-based shuffle. If push-based shuffle is turned-off then the users will not see any change.

### 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](https://issues.apache.org/jira/browse/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 #30312 from otterc/SPARK-32917.

Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Min Shen <mshen@linked.in.com>
Co-authored-by: Ye Zhou <yezhou@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-01-08 12:21:56 -06:00
Chandni Singh 0677c39009 [SPARK-32916][SHUFFLE][TEST-MAVEN][TEST-HADOOP2.7] Ensure the number of chunks in meta file and index file are equal
### What changes were proposed in this pull request?
1. Fixes for bugs in `RemoteBlockPushResolver` where the number of chunks in meta file and index file are inconsistent due to exceptions while writing to either index file or meta file. This java class was introduced in https://github.com/apache/spark/pull/30062.
 - If the writing to index file fails, the position of meta file is not reset. This means that the number of chunks in meta file is inconsistent with index file.
- During the exception handling while writing to index/meta file, we just set the pointer to the start position. If the files are closed just after this then it doesn't get rid of any the extra bytes written to it.
2. Adds an IOException threshold. If the `RemoteBlockPushResolver` encounters IOExceptions greater than this threshold  while updating data/meta/index file of a shuffle partition, then it responds to the client with  exception- `IOExceptions exceeded the threshold` so that client can stop pushing data for this shuffle partition.
3. When the update to metadata fails, exception is not propagated back to the client. This results in the increased size of the current chunk. However, with (2) in place, the current chunk will still be of a manageable size.

### Why are the changes needed?
This fix is needed for the bugs mentioned above.
1. Moved writing to meta file after index file. This fixes the issue because if there is an exception writing to meta file, then the index file position is not updated. With this change, if there is an exception writing to index file, then none of the files are effectively updated and the same is true vice-versa.
2. Truncating the lengths of data/index/meta files when the partition is finalized.
3. When the IOExceptions have reached the threshold, it is most likely that future blocks will also face the issue. So, it is better to let the clients know so that they can stop pushing the blocks for that partition.
4. When just the meta update fails, client retries pushing the block which was successfully merged to data file. This can be avoided by letting the chunk grow slightly.

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

### How was this patch tested?
Added unit tests for all the bugs and threshold.

Closes #30433 from otterc/SPARK-32916-followup.

Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-12-23 12:42:18 -06:00
Dongjoon Hyun de9818f043
[SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT
### What changes were proposed in this pull request?

This PR aims to update `master` branch version to 3.2.0-SNAPSHOT.

### Why are the changes needed?

Start to prepare Apache Spark 3.2.0.

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

N/A.

### How was this patch tested?

Pass the CIs.

Closes #30606 from dongjoon-hyun/SPARK-3.2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-12-04 14:10:42 -08:00
Josh Soref 13fd272cd3 Spelling r common dev mlib external project streaming resource managers python
### What changes were proposed in this pull request?

This PR intends to fix typos in the sub-modules:
* `R`
* `common`
* `dev`
* `mlib`
* `external`
* `project`
* `streaming`
* `resource-managers`
* `python`

Split per srowen https://github.com/apache/spark/pull/30323#issuecomment-728981618

NOTE: The misspellings have been reported at 706a726f87 (commitcomment-44064356)

### Why are the changes needed?

Misspelled words make it harder to read / understand content.

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

There are various fixes to documentation, etc...

### How was this patch tested?

No testing was performed

Closes #30402 from jsoref/spelling-R_common_dev_mlib_external_project_streaming_resource-managers_python.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-11-27 10:22:45 -06:00
angerszhu dd32f45d20 [SPARK-31069][CORE] Avoid repeat compute chunksBeingTransferred cause hight cpu cost in external shuffle service when maxChunksBeingTransferred use default value
### What changes were proposed in this pull request?
Followup from #27831 , origin author chrysan.

Each request it will check `chunksBeingTransferred `
```
public long chunksBeingTransferred() {
    long sum = 0L;
    for (StreamState streamState: streams.values()) {
      sum += streamState.chunksBeingTransferred.get();
    }
    return sum;
  }
```
  such as
```
long chunksBeingTransferred = streamManager.chunksBeingTransferred();
    if (chunksBeingTransferred >= maxChunksBeingTransferred) {
      logger.warn("The number of chunks being transferred {} is above {}, close the connection.",
        chunksBeingTransferred, maxChunksBeingTransferred);
      channel.close();
      return;
    }
```
It will  traverse `streams` repeatedly and we know that fetch data chunk will access `stream` too,  there cause two problem:

1. repeated traverse `streams`, the longer the length, the longer the time
2. lock race in ConcurrentHashMap `streams`

In this PR, when `maxChunksBeingTransferred` use default value, we avoid compute `chunksBeingTransferred ` since we don't  care about this.  If user want to set this configuration and meet performance problem,  you can also backport PR #27831

### Why are the changes needed?
Speed up  getting `chunksBeingTransferred`  and avoid lock race in object `streams`

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

### How was this patch tested?
Existed UT

Closes #30139 from AngersZhuuuu/SPARK-31069.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: chrysan <chrysanxia@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-11-17 20:52:58 -06:00
Chandni Singh 8113c88542 [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode
### What changes were proposed in this pull request?
This is one of the patches for SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602) which is needed for push-based shuffle.
Summary of changes:
- Adds an implementation of `MergedShuffleFileManager` which was introduced with [Spark 32915](https://issues.apache.org/jira/browse/SPARK-32915).
- Integrated the push-based shuffle service with `YarnShuffleService`.

### Why are the changes needed?
Refer to the SPIP in  [SPARK-30602](https://issues.apache.org/jira/browse/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](https://issues.apache.org/jira/browse/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 #30062 from otterc/SPARK-32916.

Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Ye Zhou <yezhou@linkedin.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-11-09 11:00:52 -06: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
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
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
Sean Owen 61b7d446b3 Apply appropriate RPC handler to receive, receiveStream when auth enabled 2020-04-17 13:25:12 -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
Yuanjian Li 0fe203e703 [SPARK-30623][CORE] Spark external shuffle allow disable of separate event loop group
### What changes were proposed in this pull request?
Fix the regression caused by #22173.
The original PR changes the logic of handling `ChunkFetchReqeust` from async to sync, that's causes the shuffle benchmark regression. This PR fixes the regression back to the async mode by reusing the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent`.
When the user sets the config, ChunkFetchReqeust will be processed in a separate event loop group, otherwise, the code path is exactly the same as before.

### Why are the changes needed?
Fix the shuffle performance regression described in  https://github.com/apache/spark/pull/22173#issuecomment-572459561

### Does this PR introduce any user-facing change?
Yes, this PR disable the separate event loop for FetchRequest by default.

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

Closes #27665 from xuanyuanking/SPARK-24355-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-26 12:37:48 +08: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
Chandni Singh 6b47ace27d [SPARK-30512] Added a dedicated boss event loop group
### What changes were proposed in this pull request?
Adding a dedicated boss event loop group to the Netty pipeline in the External Shuffle Service to avoid the delay in channel registration.
```
   EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1,
      conf.getModuleName() + "-boss");
    EventLoopGroup workerGroup =  NettyUtils.createEventLoop(ioMode, conf.serverThreads(),
    conf.getModuleName() + "-server");

    bootstrap = new ServerBootstrap()
      .group(bossGroup, workerGroup)
      .channel(NettyUtils.getServerChannelClass(ioMode))
      .option(ChannelOption.ALLOCATOR, allocator)
```

### Why are the changes needed?
We have been seeing a large number of SASL authentication (RPC requests) timing out with the external shuffle service.
```
java.lang.RuntimeException: java.util.concurrent.TimeoutException: Timeout waiting for task.
	at org.spark-project.guava.base.Throwables.propagate(Throwables.java:160)
	at org.apache.spark.network.client.TransportClient.sendRpcSync(TransportClient.java:278)
	at org.apache.spark.network.sasl.SaslClientBootstrap.doBootstrap(SaslClientBootstrap.java:80)
	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:228)
	at org.apache.spark.network.client.TransportClientFactory.createUnmanagedClient(TransportClientFactory.java:181)
	at org.apache.spark.network.shuffle.ExternalShuffleClient.registerWithShuffleServer(ExternalShuffleClient.java:141)
	at org.apache.spark.storage.BlockManager$$anonfun$registerWithExternalShuffleServer$1.apply$mcVI$sp(BlockManager.scala:218)
```
The investigation that we have done is described here:
https://github.com/netty/netty/issues/9890

After adding `LoggingHandler` to the netty pipeline, we saw that the registration of the channel was getting delay which is because the worker threads are busy with the existing channels.

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

### How was this patch tested?
We have tested the patch on our clusters and with a stress testing tool. After this change, we didn't see any SASL requests timing out. Existing unit tests pass.

Closes #27240 from otterc/SPARK-30512.

Authored-by: Chandni Singh <chsingh@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-29 15:02:48 -06:00
Maxim Kolesnikov 830e635e67 [SPARK-27868][CORE][FOLLOWUP] Recover the default value to -1 again
The default value for backLog set back to -1, as any other value may break existing configuration by overriding Netty's default io.netty.util.NetUtil#SOMAXCONN. The documentation accordingly adjusted.
See discussion thread: https://github.com/apache/spark/pull/24732

### What changes were proposed in this pull request?
Partial rollback of https://github.com/apache/spark/pull/24732 (default for backLog set back to -1).

### Why are the changes needed?
Previous change introduces backward incompatibility by overriding default of Netty's `io.netty.util.NetUtil#SOMAXCONN`

Closes #27230 from xCASx/master.

Authored-by: Maxim Kolesnikov <swe.kolesnikov@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-17 10:43:47 -08:00
Henrique Goulart d42cf4566a [SPARK-30246][CORE] OneForOneStreamManager might leak memory in connectionTerminated
### What changes were proposed in this pull request?

Ensure that all StreamStates are removed from OneForOneStreamManager memory map even if there's an error trying to release buffers

### Why are the changes needed?

OneForOneStreamManager may not remove all StreamStates from memory map when a connection is terminated. A RuntimeException might be thrown in StreamState$buffers.next() by one of ExternalShuffleBlockResolver$getBlockData... **breaking the loop through streams.entrySet(), keeping StreamStates in memory forever leaking memory.**
That may happen when an application is terminated abruptly and executors removed before the connection is terminated or if shuffleIndexCache fails to get ShuffleIndexInformation

References:
ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java (L319)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java (L357)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java (L195)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java (L208)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java (L330)

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

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

Closes #27064 from hensg/SPARK-30246.

Lead-authored-by: Henrique Goulart <henriquedsg89@gmail.com>
Co-authored-by: Henrique Goulart <henrique.goulart@trivago.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 13:27:15 -08:00
Ajith 700293207d [SPARK-30406] OneForOneStreamManager ensure that compound operations on shared variables are atomic
Using compound operations as well as increments and decrements on primitive fields are not atomic operations. Here when volatile primitive field is incremented or decremented,  we run into data loss if threads interleave in steps of update.

 Refer: https://wiki.sei.cmu.edu/confluence/display/java/VNA02-J.+Ensure+that+compound+operations+on+shared+variables+are+atomic

### What changes were proposed in this pull request?
Using `AtomicLong` instead of `long`

### Why are the changes needed?
volatile primitive field is incremented or decremented,  we run into data loss if threads interleave in steps of update.

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

### How was this patch tested?
All Existing UT can pass with the Change

Closes #27071 from ajithme/atomic.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-03 11:41:45 -06:00
Sean Owen 7dff3b125d [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes
### What changes were proposed in this pull request?

Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.

### Why are the changes needed?

Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.

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

Should not be, but see notes below on hash codes and toString.

### How was this patch tested?

Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.

Closes #26911 from srowen/SPARK-30272.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:55:04 -06:00
Yuming Wang 696288f623 [INFRA] Reverts commit 56dcd79 and c216ef1
### What changes were proposed in this pull request?
1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79

2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1

### Why are the changes needed?
Shouldn't change master.

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

### How was this patch tested?
manual test:
https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master

Closes #26915 from wangyum/revert-master.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-12-16 19:57:44 -07:00
Yuming Wang 56dcd79992 Preparing development version 3.0.1-SNAPSHOT 2019-12-17 01:57:27 +00:00
Yuming Wang c216ef1d03 Preparing Spark release v3.0.0-preview2-rc2 2019-12-17 01:57:21 +00:00
Marcelo Vanzin c5f312a6ac [SPARK-30129][CORE] Set client's id in TransportClient after successful auth
The new auth code was missing this bit, so it was not possible to know which
app a client belonged to when auth was on.

I also refactored the SASL test that checks for this so it also checks the
new protocol (test failed before the fix, passes now).

Closes #26760 from vanzin/SPARK-30129.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-04 17:11:50 -08:00
huangtianhua e842033acc [SPARK-27721][BUILD] Switch to use right leveldbjni according to the platforms
This change adds a profile to switch to use the right leveldbjni package according to the platforms:
aarch64 uses org.openlabtesting.leveldbjni:leveldbjni-all.1.8, and other platforms use the old one org.fusesource.leveldbjni:leveldbjni-all.1.8.
And because some hadoop dependencies packages are also depend on org.fusesource.leveldbjni:leveldbjni-all, but hadoop merge the similar change on trunk, details see
https://issues.apache.org/jira/browse/HADOOP-16614, so exclude the dependency of org.fusesource.leveldbjni for these hadoop packages related.
Then Spark can build/test on aarch64 platform successfully.

Closes #26636 from huangtianhua/add-aarch64-leveldbjni.

Authored-by: huangtianhua <huangtianhua@huawei.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-12-02 09:04:00 -06:00
Norman Maurer f28eab2de7 [SPARK-29971][CORE] Fix buffer leaks in TransportFrameDecoder/TransportCipher
### What changes were proposed in this pull request?

- Correctly release `ByteBuf` in `TransportCipher` in all cases
- Move closing / releasing logic to `handlerRemoved(...)` so we are guaranteed that is always called.
- Correctly release `frameBuf` it is not null when the handler is removed (and so also when the channel becomes inactive)

### Why are the changes needed?

We need to carefully manage the ownership / lifecycle of `ByteBuf` instances so we don't leak any of these. We did not correctly do this in all cases:
 - when end up in invalid cipher state.
 - when partial data was received and the channel is closed before the full frame is decoded

Fixes https://github.com/netty/netty/issues/9784.

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

No.

### How was this patch tested?

Pass the newly added UTs.

Closes #26609 from normanmaurer/fix_leaks.

Authored-by: Norman Maurer <norman_maurer@apple.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-22 15:20:54 -08:00
Maxim Gekk 80a89873b2 [SPARK-29733][TESTS] Fix wrong order of parameters passed to assertEquals
### What changes were proposed in this pull request?
The `assertEquals` method of JUnit Assert requires the first parameter to be the expected value. In this PR, I propose to change the order of parameters when the expected value is passed as the second parameter.

### Why are the changes needed?
Wrong order of assert parameters confuses when the assert fails and the parameters have special string representation. For example:
```java
assertEquals(input1.add(input2), new CalendarInterval(5, 5, 367200000000L));
```
```
java.lang.AssertionError:
Expected :interval 5 months 5 days 101 hours
Actual   :interval 5 months 5 days 102 hours
```

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

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

Closes #26377 from MaxGekk/fix-order-in-assert-equals.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 11:21:28 -08:00
Xingbo Jiang 8207c835b4 Revert "Prepare Spark release v3.0.0-preview-rc2"
This reverts commit 007c873ae3.
2019-10-30 17:45:44 -07:00
Xingbo Jiang 007c873ae3 Prepare Spark release v3.0.0-preview-rc2
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the sparkR version number check logic to allow jvm version like `3.0.0-preview`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

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

No

### How was this patch tested?

N/A
2019-10-30 17:42:59 -07:00
Xingbo Jiang b33a58c0c6 Revert "Prepare Spark release v3.0.0-preview-rc1"
This reverts commit 5eddbb5f1d.
2019-10-28 22:32:34 -07:00
Xingbo Jiang 5eddbb5f1d Prepare Spark release v3.0.0-preview-rc1
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the PySpark version from `3.0.0.dev0` to `3.0.0`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

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

No

### How was this patch tested?

N/A

Closes #26243 from jiangxb1987/3.0.0-preview-prepare.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-28 22:31:29 -07:00
Yuanjian Li f725d472f5 [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files
After the newly added shuffle block fetching protocol in #24565, we can keep this work by extending the FetchShuffleBlocks message.

### What changes were proposed in this pull request?
In this patch, we achieve the indeterminate shuffle rerun by reusing the task attempt id(unique id within an application) in shuffle id, so that each shuffle write attempt has a different file name. For the indeterministic stage, when the stage resubmits, we'll clear all existing map status and rerun all partitions.

All changes are summarized as follows:
- Change the mapId to mapTaskAttemptId in shuffle related id.
- Record the mapTaskAttemptId in MapStatus.
- Still keep mapId in ShuffleFetcherIterator for fetch failed scenario.
- Add the determinate flag in Stage and use it in DAGScheduler and the cleaning work for the intermediate stage.

### Why are the changes needed?
This is a follow-up work for #22112's future improvment[1]: `Currently we can't rollback and rerun a shuffle map stage, and just fail.`

Spark will rerun a finished shuffle write stage while meeting fetch failures, currently, the rerun shuffle map stage will only resubmit the task for missing partitions and reuse the output of other partitions. This logic is fine in most scenarios, but for indeterministic operations(like repartition), multiple shuffle write attempts may write different data, only rerun the missing partition will lead a correctness bug. So for the shuffle map stage of indeterministic operations, we need to support rolling back the shuffle map stage and re-generate the shuffle files.

### Does this PR introduce any user-facing change?
Yes, after this PR, the indeterminate stage rerun will be accepted by rerunning the whole stage. The original behavior is aborting the stage and fail the job.

### How was this patch tested?
- UT: Add UT for all changing code and newly added function.
- Manual Test: Also providing a manual test to verify the effect.
```
import scala.sys.process._
import org.apache.spark.TaskContext

val determinateStage0 = sc.parallelize(0 until 1000 * 1000 * 100, 10)
val indeterminateStage1 = determinateStage0.repartition(200)
val indeterminateStage2 = indeterminateStage1.repartition(200)
val indeterminateStage3 = indeterminateStage2.repartition(100)
val indeterminateStage4 = indeterminateStage3.repartition(300)
val fetchFailIndeterminateStage4 = indeterminateStage4.map { x =>
if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId == 190 &&
  TaskContext.get.stageAttemptNumber == 0) {
  throw new Exception("pkill -f -n java".!!)
  }
  x
}
val indeterminateStage5 = fetchFailIndeterminateStage4.repartition(200)
val finalStage6 = indeterminateStage5.repartition(100).collect().distinct.length
```
It's a simple job with multi indeterminate stage, it will get a wrong answer while using old Spark version like 2.2/2.3, and will be killed after #22112. With this fix, the job can retry all indeterminate stage as below screenshot and get the right result.
![image](https://user-images.githubusercontent.com/4833765/63948434-3477de00-caab-11e9-9ed1-75abfe6d16bd.png)

Closes #25620 from xuanyuanking/SPARK-25341-8.27.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-09-23 16:16:52 +08:00
colinma 076186e881 [SPARK-19147][CORE] Gracefully handle error in task after executor is stopped
### What changes were proposed in this pull request?

TransportClientFactory.createClient() is called by task and TransportClientFactory.close() is called by executor.
When stop the executor, close() will set workerGroup = null, NPE will occur in createClient which generate many exception in log.
For exception occurs after close(), treated it as an expected Exception
and transform it to InterruptedException which can be processed by Executor.

### Why are the changes needed?

The change can reduce the exception stack trace in log file, and user won't be confused by these excepted exception.

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

N/A

### How was this patch tested?

New tests are added in TransportClientFactorySuite and ExecutorSuite

Closes #25759 from colinmjj/spark-19147.

Authored-by: colinma <colinma@tencent.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-21 07:31:39 -05:00
Dongjoon Hyun 471a3eff51 [SPARK-28932][BUILD][FOLLOWUP] Switch to scala-library compile dependency for JDK11
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/25638 to switch `scala-library` from `test` dependency to `compile` dependency in `network-common` module.

### Why are the changes needed?

Previously, we added `scala-library` as a test dependency to resolve the followings, but it was insufficient to resolve. This PR aims to switch it to compile dependency.
```
$ java -version
openjdk version "11.0.3" 2019-04-16
OpenJDK Runtime Environment AdoptOpenJDK (build 11.0.3+7)
OpenJDK 64-Bit Server VM AdoptOpenJDK (build 11.0.3+7, mixed mode)

$ mvn clean install -pl common/network-common -DskipTests
...
[INFO] --- scala-maven-plugin:4.2.0:doc-jar (attach-scaladocs)  spark-network-common_2.12 ---
error: fatal error: object scala in compiler mirror not found.
one error found
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
```

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

No.

### How was this patch tested?

Manually, run the following on JDK11.
```
$ mvn clean install -pl common/network-common -DskipTests
```

Closes #25800 from dongjoon-hyun/SPARK-28932-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-16 00:13:07 -07:00
Dongjoon Hyun 1f96ce5443 [SPARK-28932][BUILD] Add scala-library test dependency to network-common module for JDK11
### What changes were proposed in this pull request?

This PR adds `scala-library` test dependency to `network-common` module for JDK11.

### Why are the changes needed?

In JDK11, the following command fails due to scala library.
```
mvn clean install -pl common/network-common -DskipTests
```

**BEFORE**
```
...
error: fatal error: object scala in compiler mirror not found.
one error found
...
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
[INFO] ------------------------------------------------------------------------
```

**AFTER**
```
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
```

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

No.

### How was this patch tested?

Manual. On JDK11, do the following.
```
mvn clean install -pl common/network-common -DskipTests
```

Closes #25638 from dongjoon-hyun/SPARK-28932.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-31 10:59:20 -07:00
younggyu chun 8535df7261 [MINOR] Fix typos in comments and replace an explicit type with <>
## What changes were proposed in this pull request?
This PR fixed typos in comments and replace the explicit type with '<>' for Java 8+.

## How was this patch tested?
Manually tested.

Closes #25338 from younggyuchun/younggyu.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-10 16:47:11 -05:00
Gabor Somogyi e11a55827e [SPARK-28261][CORE] Fix client reuse test
## What changes were proposed in this pull request?

There is the following code in [TransportClientFactory#createClient](https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java#L150)
```
    int clientIndex = rand.nextInt(numConnectionsPerPeer);
    TransportClient cachedClient = clientPool.clients[clientIndex];
```
which choose a client from its pool randomly. If we are unlucky we might not get the max number of connections out, but less than that.

To prove that I've tried out the following test:
```java
  Test
  public void testRandom() {
    Random rand = new Random();
    Set<Integer> clients = Collections.synchronizedSet(new HashSet<>());
    long iterCounter = 0;
    while (true) {
      iterCounter++;
      int maxConnections = 4;
      clients.clear();
      for (int i = 0; i < maxConnections * 10; i++) {
        int clientIndex = rand.nextInt(maxConnections);
        clients.add(clientIndex);
      }
      if (clients.size() != maxConnections) {
        System.err.println("Unexpected clients size (iterCounter=" + iterCounter + "): " + clients.size() + ", maxConnections: " + maxConnections);
      }
      if (iterCounter % 100000 == 0) {
        System.out.println("IterCounter: " + iterCounter);
      }
    }
  }
```

Result:
```
Unexpected clients size (iterCounter=22388): 3, maxConnections: 4
Unexpected clients size (iterCounter=36244): 3, maxConnections: 4
Unexpected clients size (iterCounter=85798): 3, maxConnections: 4
IterCounter: 100000
Unexpected clients size (iterCounter=97108): 3, maxConnections: 4
Unexpected clients size (iterCounter=119121): 3, maxConnections: 4
Unexpected clients size (iterCounter=129948): 3, maxConnections: 4
Unexpected clients size (iterCounter=173736): 3, maxConnections: 4
Unexpected clients size (iterCounter=178138): 3, maxConnections: 4
Unexpected clients size (iterCounter=195108): 3, maxConnections: 4
IterCounter: 200000
Unexpected clients size (iterCounter=209006): 3, maxConnections: 4
Unexpected clients size (iterCounter=217105): 3, maxConnections: 4
Unexpected clients size (iterCounter=222456): 3, maxConnections: 4
Unexpected clients size (iterCounter=226899): 3, maxConnections: 4
Unexpected clients size (iterCounter=229101): 3, maxConnections: 4
Unexpected clients size (iterCounter=253549): 3, maxConnections: 4
Unexpected clients size (iterCounter=277550): 3, maxConnections: 4
Unexpected clients size (iterCounter=289637): 3, maxConnections: 4
...
```

In this PR I've adapted the test code not to have this flakyness.

## How was this patch tested?

Additional (not committed test) + existing unit tests in a loop.

Closes #25075 from gaborgsomogyi/SPARK-28261.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-08 11:10:03 -07:00
LantaoJin 0e421000e0 [SPARK-28160][CORE] Fix a bug that callback function may hang when unchecked exception missed
## What changes were proposed in this pull request?

This is very like #23590 .

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the response is large but no enough memory is available. However, when this happens, `TransportClient.sendRpcSync` will just hang forever if the timeout set to unlimited.

This PR catches `Throwable` and uses the error to complete `SettableFuture`.

## How was this patch tested?

I tested in my IDE by setting the value of size to -1 to verify the result. Without this patch, it won't be finished until timeout (May hang forever if timeout set to MAX_INT), or the expected `IllegalArgumentException` will be caught.
```java
Override
      public void onSuccess(ByteBuffer response) {
        try {
          int size = response.remaining();
          ByteBuffer copy = ByteBuffer.allocate(size); // set size to -1 in runtime when debug
          copy.put(response);
          // flip "copy" to make it readable
          copy.flip();
          result.set(copy);
        } catch (Throwable t) {
          result.setException(t);
        }
      }
```

Closes #24964 from LantaoJin/SPARK-28160.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: lajin <lajin@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-30 15:14:41 -05:00
Sean Owen 9df7587eea [MINOR][CORE] Fix line too long in TransportClientFactory
## What changes were proposed in this pull request?

See https://github.com/apache/spark/pull/24702/files#r296765487 -- this just fixes a Java style error. I'm not clear why the PR build didn't catch it.

## How was this patch tested?

N/A

Closes #24951 from srowen/SPARK-27989.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 11:17:54 -05:00
Jose Luis Pedrosa 0671395966 [SPARK-27989][CORE] Added retries on the connection to the driver for k8s
Disabled negative dns caching for docker images

Improved logging on DNS resolution, convenient for slow k8s clusters

## What changes were proposed in this pull request?
Added retries when building the connection to the driver in K8s.
In some scenarios DNS reslution can take more than the timeout.
Also openjdk-8 by default has negative dns caching enabled, which means even retries may not help depending on the times.

## How was this patch tested?
This patch was tested agains an specific k8s cluster with slow response time in DNS to ensure it woks.

Closes #24702 from jlpedrosa/feature/kuberetries.

Authored-by: Jose Luis Pedrosa <jlpedrosa@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 09:25:43 -05:00
Marcelo Vanzin 09ed64d795 [SPARK-27868][CORE] Better default value and documentation for socket server backlog.
First, there is currently no public documentation for this setting. So it's hard
to even know that it could be a problem if your application starts failing with
weird shuffle errors.

Second, the javadoc attached to the code was incorrect; the default value just uses
the default value from the JRE, which is 50, instead of having an unbounded queue
as the comment implies.

So use a default that is a "rounded" version of the JRE default, and provide
documentation explaining that this value may need to be adjusted. Also added
a log message that was very helpful in debugging an issue caused by this
problem.

Closes #24732 from vanzin/SPARK-27868.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-29 14:56:36 -07:00
Shixiong Zhu 04f142db9c
[SPARK-20547][REPL] Throw RemoteClassLoadedError for transient errors in ExecutorClassLoader
## What changes were proposed in this pull request?

`ExecutorClassLoader`'s `findClass` may fail to fetch a class due to transient exceptions. For example, when a task is interrupted, if `ExecutorClassLoader` is fetching a class, you may see `InterruptedException` or `IOException` wrapped by `ClassNotFoundException`, even if this class can be loaded. Then the result of `findClass` will be cached by JVM, and later when the same class is being loaded in the same executor, it will just throw NoClassDefFoundError even if the class can be loaded.

I found JVM only caches `LinkageError` and `ClassNotFoundException`. Hence in this PR, I changed ExecutorClassLoader to throw `RemoteClassLoadedError` if we cannot get a response from driver.

## How was this patch tested?

New unit tests.

Closes #24683 from zsxwing/SPARK-20547-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-05-28 12:56:14 -07:00
Yuanjian Li 8949bc7a3c [SPARK-27665][CORE] Split fetch shuffle blocks protocol from OpenBlocks
## What changes were proposed in this pull request?

As the current approach in OneForOneBlockFetcher, we reuse the OpenBlocks protocol to describe the fetch request for shuffle blocks, and it causes the extension work for shuffle fetching like #19788 and #24110 very awkward.
In this PR, we split the fetch request for shuffle blocks from OpenBlocks which named FetchShuffleBlocks. It's a loose bind with ShuffleBlockId and can easily extend by adding new fields in this protocol.

## How was this patch tested?

Existing and new added UT.

Closes #24565 from xuanyuanking/SPARK-27665.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-27 22:19:31 +08:00
“attilapiros” e9f3f62b2c [SPARK-27677][CORE] Serve local disk persisted blocks by the external service after releasing executor by dynamic allocation
# What changes were proposed in this pull request?

## Problem statement

An executor which has persisted blocks does not consider to be idle and this way ready to be released by dynamic allocation after the regular timeout `spark.dynamicAllocation.executorIdleTimeout` but there is separate configuration `spark.dynamicAllocation.cachedExecutorIdleTimeout` which defaults to `Integer.MAX_VALUE`. This is because releasing the executor also means losing the persisted blocks (as the metadata for individual blocks called `BlockInfo` are kept in memory) and when the RDD is referenced latter on this lost blocks will be recomputed.
On the other hand keeping the executors too long without any task to work on is also a waste of resources (as executors are reserved for the application by the resource manager).

## Solution

This PR focuses on the first part of SPARK-25888: it extends the external shuffle service with the capability to serve RDD blocks which are persisted on the local disk store by the executors. Moreover when this feature is enabled by setting the `spark.shuffle.service.fetch.rdd.enabled` config to true and a block is reported to be persisted on to disk the external shuffle service instance running on the same host as the executor is also registered (along with the reporting block manager) as a possible location for fetching it.

## Some implementation detail

Some explanation about the decisions made during the development:
- the location list to fetch a block was randomized but the groups (same host, same rack, others) order was kept. In this PR the order of groups are kept and external shuffle service added to the end of the each group.
- `BlockManagerInfo` is not introduced for external shuffle service but only a lightweight solution is taken. A hash map from `BlockId` to `BlockStatus` is introduced. A type alias would make the source more readable but I know it is discouraged. On the other hand a new class wrapping this hash map would introduce unnecessary indirection.
- when this feature is on the cleanup triggered during removing of executors (which is handled in `ExternalShuffleBlockResolver`) is modified to keep the disk persisted RDD blocks. This cleanup is triggered in standalone mode when the `spark.storage.cleanupFilesAfterExecutorExit` config is set.
- the unpersisting of an RDD is extended to use the external shuffle service for disk persisted RDD blocks when the original executor which created the blocks are already released. New block transport messages are introduced to support this: `RemoveBlocks` and `BlocksRemoved`.

# How was this patch tested?

## Unit tests

### ExternalShuffleServiceSuite

Here the complete use case is tested by the "SPARK-25888: using external shuffle service fetching disk persisted blocks" with a tiny difference: here the executor is killed manually, this way the test is a bit faster than waiting for the idle timeout.

### ExternalShuffleBlockHandlerSuite

Tests the fetching of the RDD blocks via the external shuffle service.

### BlockManagerInfoSuite

This a new suite. As the `BlockManagerInfo` behaviour depends very much on whether the external shuffle service enabled or not all the tests are executed with and without it.

### BlockManagerSuite

Tests the sorting of the block locations.

## Manually on YARN

Spark App was:

~~~scala
package com.mycompany

import org.apache.spark.rdd.RDD
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.storage.StorageLevel

object TestAppDiskOnlyLevel {
  def main(args: Array[String]): Unit = {
    val conf = new SparkConf().setAppName("test-app")

    println("Attila: START")
    val sc = new SparkContext(conf)
    val rdd = sc.parallelize(0 until 100, 10)
      .map { i =>
        println(s"Attila: calculate first rdd i=$i")
        Thread.sleep(1000)
        i
      }

    rdd.persist(StorageLevel.DISK_ONLY)
    rdd.count()

    println("Attila: First RDD is processed, waiting for 60 sec")

    Thread.sleep(60 * 1000)

    println("Attila: Num executors must be 0 as executorIdleTimeout is way over")

    val rdd2 = sc.parallelize(0 until 10, 1)
      .map(i => (i, 1))
      .persist(StorageLevel.DISK_ONLY)

    rdd2.count()

    println("Attila: Second RDD with one partition (only one executors must be alive)")

    // reduce runs as user code to detect the empty seq (empty blocks)
    println("Calling collect on the first RDD: " + rdd.collect().reduce(_ + _))

    println("Attila: STOP")
  }
}
~~~

I have submitted with the following configuration:

~~~bash
spark-submit --master yarn \
  --conf spark.dynamicAllocation.enabled=true \
  --conf spark.dynamicAllocation.executorIdleTimeout=30 \
  --conf spark.dynamicAllocation.cachedExecutorIdleTimeout=90 \
  --class com.mycompany.TestAppDiskOnlyLevel dyn_alloc_demo-core_2.11-0.1.0-SNAPSHOT-jar-with-dependencies.jar
~~~

Checked the result by filtering for the side effect of the task calculations:

~~~bash
[userserver ~]$ yarn logs -applicationId application_1556299359453_0001 | grep "Attila: calculate" | wc -l
WARNING: YARN_OPTS has been replaced by HADOOP_OPTS. Using value of YARN_OPTS.
19/04/26 10:31:59 INFO client.RMProxy: Connecting to ResourceManager at apiros-1.gce.company.com/172.31.115.165:8032
100
~~~

So it is only 100 task execution and not 200 (which would be the case for re-computation).

Moreover from the submit/launcher log we can see executors really stopped in between (see the new total is 0 before the last line):
~~~
[userserver ~]$ grep "Attila: Num executors must be 0" -B 2 spark-submit.log
19/04/26 10:24:27 INFO cluster.YarnScheduler: Executor 9 on apiros-3.gce.company.com killed by driver.
19/04/26 10:24:27 INFO spark.ExecutorAllocationManager: Existing executor 9 has been removed (new total is 0)
Attila: Num executors must be 0 as executorIdleTimeout is way over
~~~

[Full spark submit log](https://github.com/attilapiros/spark/files/3122465/spark-submit.log)

I have done a test also after changing the `DISK_ONLY` storage level to `MEMORY_ONLY` for the first RDD. After this change during the 60sec waiting no executor was removed.

Closes #24499 from attilapiros/SPARK-25888-final.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-23 13:15:39 -07:00