Commit graph

185 commits

Author SHA1 Message Date
Gengliang Wang 1bad04d028 Preparing development version 3.2.1-SNAPSHOT 2021-08-31 17:04:14 +00:00
Gengliang Wang 03f5d23e96 Preparing Spark release v3.2.0-rc2 2021-08-31 17:04:08 +00:00
Venkata krishnan Sowrirajan 0f2e318894 [SPARK-36374][FOLLOW-UP] Change config key spark.shuffle.server.mergedShuffleFileManagerImpl to spark.shuffle.push.server.mergedShuffleFileManagerImpl
### What changes were proposed in this pull request?

Minor changes to change the config key name from `spark.shuffle.server.mergedShuffleFileManagerImpl` to `spark.shuffle.push.server.mergedShuffleFileManagerImpl`. This is missed out in https://github.com/apache/spark/pull/33615.

### Why are the changes needed?

To keep the config names consistent

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

Yes, this is a change in the config key name. But the new config name changes are yet to be released. Technically there is no user facing change because of this change.

### How was this patch tested?

Existing tests.

Closes #33799 from venkata91/SPARK-36374-follow-up.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 7b2842e986)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-22 01:29:36 -05:00
sweisdb 243bfafd5c Updates AuthEngine to pass the correct SecretKeySpec format
AuthEngineSuite was passing on some platforms (MacOS), but failing on others (Linux) with an InvalidKeyException stemming from this line. We should explicitly pass AES as the key format.

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

Changes the AuthEngine SecretKeySpec from "RAW" to "AES".

### Why are the changes needed?

Unit tests were failing on some platforms with InvalidKeyExceptions when this key was used to instantiate a Cipher.

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

No.

### How was this patch tested?

Unit tests on a MacOS and Linux platform.

Closes #33790 from sweisdb/patch-1.

Authored-by: sweisdb <60895808+sweisdb@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit c441c7e365)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-08-20 08:31:54 -05:00
Gengliang Wang 69be513c5e Preparing development version 3.2.1-SNAPSHOT 2021-08-20 12:40:47 +00:00
Gengliang Wang 6bb3523d8e Preparing Spark release v3.2.0-rc1 2021-08-20 12:40:40 +00:00
Gengliang Wang fafdc1482b Revert "Preparing Spark release v3.2.0-rc1"
This reverts commit 8e58fafb05.
2021-08-20 20:07:02 +08:00
Gengliang Wang c829ed53ff Revert "Preparing development version 3.2.1-SNAPSHOT"
This reverts commit 4f1d21571d.
2021-08-20 20:07:01 +08:00
Gengliang Wang 4f1d21571d Preparing development version 3.2.1-SNAPSHOT 2021-08-19 14:08:32 +00:00
Gengliang Wang 8e58fafb05 Preparing Spark release v3.2.0-rc1 2021-08-19 14:08:26 +00:00
zhuqi-lucas 2fb62e0e3d [SPARK-35548][CORE][SHUFFLE] Handling new attempt has started error message in BlockPushErrorHandler in client
### What changes were proposed in this pull request?
Add a new type of error message in BlockPushErrorHandler which indicates the PushblockStream message is received after a new application attempt has started. This error message should be correctly handled in client without retrying the block push.

### Why are the changes needed?
When we get a block push failure because of the too old attempt, we will not retry pushing the block nor log the exception on the client side.

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

### How was this patch tested?
Add the corresponding unit test.

Closes #33617 from zhuqi-lucas/master.

Authored-by: zhuqi-lucas <821684824@qq.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 05cd5f97c3)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-16 13:59:54 -05:00
Venkata krishnan Sowrirajan 233af3d239 [SPARK-36374][SHUFFLE][DOC] Push-based shuffle high level user documentation
### What changes were proposed in this pull request?

Document the push-based shuffle feature with a high level overview of the feature and corresponding configuration options for both shuffle server side as well as client side. This is how the changes to the doc looks on the browser ([img](https://user-images.githubusercontent.com/8871522/129231582-ad86ee2f-246f-4b42-9528-4ccd693e86d2.png))

### Why are the changes needed?

Helps users understand the feature

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

Docs

### How was this patch tested?

N/A

Closes #33615 from venkata91/SPARK-36374.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 2270ecf32f)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-16 10:25:33 -05:00
Sean Owen b8c1014e23 Update Spark key negotiation protocol 2021-08-14 09:08:29 -05:00
Min Shen 32d67d1a93 [SPARK-36483][CORE][TESTS] Fix intermittent test failures at Netty 4.1.52+
### What changes were proposed in this pull request?

Fix an intermittent test failure due to Netty dependency version bump.
Starting from Netty 4.1.52, its AbstractChannel will throw a new `StacklessClosedChannelException` for channel closed exception.
A hardcoded list of Strings to match for channel closed exception in `RPCIntegrationSuite` was not updated, thus leading to the intermittent test failure reported in #33613

### Why are the changes needed?

Fix intermittent test failure

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

No

### How was this patch tested?

Closes #33713 from Victsm/SPARK-36378-followup.

Authored-by: Min Shen <mshen@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit b8e2186fe1)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-12 20:15:58 -05:00
Min Shen c6b683e5a2 [SPARK-36378][SHUFFLE] Switch to using RPCResponse to communicate common block push failures to the client
We have run performance evaluations on the version of push-based shuffle committed to upstream so far, and have identified a few places for further improvements:
1. On the server side, we have noticed that the usage of `String.format`, especially when receiving a block push request, has a much higher overhead compared with string concatenation.
2. On the server side, the usage of `Throwables.getStackTraceAsString` in the `ErrorHandler.shouldRetryError` and `ErrorHandler.shouldLogError` has generated quite some overhead.

These 2 issues are related to how we are currently handling certain common block push failures.
We are communicating such failures via `RPCFailure` by transmitting the exception stack trace.
This generates the overhead on both server and client side for creating these exceptions and makes checking the type of failures fragile and inefficient with string matching of exception stack trace.
To address these, this PR also proposes to encode the common block push failure as an error code and send that back to the client with a proper RPC message.

Improve shuffle service efficiency for push-based shuffle.
Improve code robustness for handling block push failures.

No

Existing unit tests.

Closes #33613 from Victsm/SPARK-36378.

Lead-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Min Shen <victor.nju@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 3f09093a21)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-10 16:54:21 -05:00
Venkata krishnan Sowrirajan 1a432fe6bb [SPARK-36460][SHUFFLE] Pull out NoOpMergedShuffleFileManager inner class outside
### What changes were proposed in this pull request?

Pull out NoOpMergedShuffleFileManager inner class outside. This is required since passing dollar sign ($) for the config (`spark.shuffle.server.mergedShuffleFileManagerImpl`) value can be an issue. Currently `spark.shuffle.server.mergedShuffleFileManagerImpl` is by default set to `org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager`. After this change the default value be set to `org.apache.spark.network.shuffle.NoOpMergedShuffleFileManager`

### Why are the changes needed?

Passing `$` for the config value can be an issue.

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

No

### How was this patch tested?

Modified existing unit tests.

Closes #33688 from venkata91/SPARK-36460.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit df0de83c46)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-10 10:19:41 +08:00
Angerszhuuuu b4c065b7db [SPARK-36391][SHUFFLE] When state is remove will throw NPE, and we should improve the error message
### What changes were proposed in this pull request?
When channel terminated will call `connectionTerminated` and remove corresponding StreamState,
then all coming request on this StreamState will throw NPE like
```
2021-07-31 22:00:24,810 ERROR server.ChunkFetchRequestHandler (ChunkFetchRequestHandler.java:lambda$respond$1(146)) - Error sending result ChunkFetchFailure[streamChunkId=StreamChunkId[streamId=1119950114515,chunkIndex=0],errorString=java.lang.NullPointerException
	at org.apache.spark.network.server.OneForOneStreamManager.getChunk(OneForOneStreamManager.java:80)
	at org.apache.spark.network.server.ChunkFetchRequestHandler.processFetchRequest(ChunkFetchRequestHandler.java:101)
	at org.apache.spark.network.server.ChunkFetchRequestHandler.channelRead0(ChunkFetchRequestHandler.java:82)
	at org.apache.spark.network.server.ChunkFetchRequestHandler.channelRead0(ChunkFetchRequestHandler.java:51)
	at org.sparkproject.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:61)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:370)
	at org.sparkproject.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
	at org.sparkproject.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
	at org.sparkproject.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:500)
	at org.sparkproject.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
	at org.sparkproject.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at org.sparkproject.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
] to /ip:53818; closing connection
java.nio.channels.ClosedChannelException
	at org.sparkproject.io.netty.channel.AbstractChannel$AbstractUnsafe.newClosedChannelException(AbstractChannel.java:957)
	at org.sparkproject.io.netty.channel.AbstractChannel$AbstractUnsafe.write(AbstractChannel.java:865)
	at org.sparkproject.io.netty.channel.DefaultChannelPipeline$HeadContext.write(DefaultChannelPipeline.java:1367)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:717)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:709)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:792)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:702)
	at org.sparkproject.io.netty.handler.codec.MessageToMessageEncoder.write(MessageToMessageEncoder.java:112)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:717)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:709)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:792)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:702)
	at org.sparkproject.io.netty.handler.timeout.IdleStateHandler.write(IdleStateHandler.java:302)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:717)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:764)
	at org.sparkproject.io.netty.channel.AbstractChannelHandlerContext$WriteTask.run(AbstractChannelHandlerContext.java:1104)
	at org.sparkproject.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
	at org.sparkproject.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
	at org.sparkproject.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:497)
	at org.sparkproject.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
	at org.sparkproject.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at org.sparkproject.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
```

Since JVM will not show stack of NPE exception if it happen many times.
```
021-07-28 08:25:44,720 ERROR server.ChunkFetchRequestHandler (ChunkFetchRequestHandler.java:lambda$respond$1(146)) - Error sending result ChunkFetchFailure[streamChunkId=StreamChunkId[streamId=1187623335353,chunkIndex=11],errorString=java.lang.NullPoint
erException
] to /10.130.10.5:42148; closing connection
java.nio.channels.ClosedChannelException
```
Makes user confused.
We should improved this error message?

### Why are the changes needed?
Improve error message

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

### How was this patch tested?

Closes #33622 from AngersZhuuuu/SPARK-36391.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmaihu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
(cherry picked from commit b377ea26e2)
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-08-05 15:32:05 +08:00
Venkata krishnan Sowrirajan a9b32b390c [SPARK-32923][CORE][SHUFFLE] Handle indeterminate stage retries for push-based shuffle
[[SPARK-23243](https://issues.apache.org/jira/browse/SPARK-23243)] and [[SPARK-25341](https://issues.apache.org/jira/browse/SPARK-25341)] addressed cases of stage retries for indeterminate stage involving operations like repartition. This PR addresses the same issues in the context of push-based shuffle. Currently there is no way to distinguish the current execution of a stage for a shuffle ID. Therefore the changes explained below are necessary.

Core changes are summarized as follows:

1. Introduce a new variable `shuffleMergeId` in `ShuffleDependency` which is monotonically increasing value tracking the temporal ordering of execution of <stage-id, stage-attempt-id> for a shuffle ID.
2. Correspondingly make changes in the push-based shuffle protocol layer in `MergedShuffleFileManager`, `BlockStoreClient` passing the `shuffleMergeId` in order to keep track of the shuffle output in separate files on the shuffle service side.
3. `DAGScheduler` increments the `shuffleMergeId` tracked in `ShuffleDependency` in the cases of a indeterministic stage execution
4. Deterministic stage will have `shuffleMergeId` set to 0 as no special handling is needed in this case and indeterminate stage will have `shuffleMergeId` starting from 1.

New protocol changes are needed due to the reasons explained above.

No

Added new unit tests in `RemoteBlockPushResolverSuite, DAGSchedulerSuite, BlockIdSuite, ErrorHandlerSuite`

Closes #33034 from venkata91/SPARK-32923.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit c039d99812)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-01 23:30:02 -05:00
Erik Krogen 23e48dbf77 [SPARK-35259][SHUFFLE] Update ExternalBlockHandler Timer variables to expose correct units
### What changes were proposed in this pull request?
`ExternalBlockHandler` exposes 4 metrics which are Dropwizard `Timer` metrics, and are named with a `millis` suffix:
```
    private final Timer openBlockRequestLatencyMillis = new Timer();
    private final Timer registerExecutorRequestLatencyMillis = new Timer();
    private final Timer fetchMergedBlocksMetaLatencyMillis = new Timer();
    private final Timer finalizeShuffleMergeLatencyMillis = new Timer();
```
However these Dropwizard Timers by default use nanoseconds ([documentation](https://metrics.dropwizard.io/3.2.3/getting-started.html#timers)).

This causes `YarnShuffleServiceMetrics` to expose confusingly-named metrics like `openBlockRequestLatencyMillis_nanos_max` (the actual values are currently in nanos).

This PR adds a new `Timer` subclass, `TimerWithCustomTimeUnit`, which accepts a `TimeUnit` at creation time and exposes timing information using this time unit when values are read. Internally, values are still stored with nanosecond-level precision. The `Timer` metrics within `ExternalBlockHandler` are updated to use the new class with milliseconds as the unit. The logic to include the `nanos` suffix in the metric name within `YarnShuffleServiceMetrics` has also been removed, with the assumption that the metric name itself includes the units.

### Does this PR introduce _any_ user-facing change?
Yes, there are two changes.
First, the names for metrics exposed by `ExternalBlockHandler` via `YarnShuffleServiceMetrics` such as `openBlockRequestLatencyMillis_nanos_max` and `openBlockRequestLatencyMillis_nanos_50thPercentile` have been changed to remove the `_nanos` suffix. This would be considered a breaking change, but these names were only exposed as part of #32388, which has not yet been released (slated for 3.2.0). New names are like `openBlockRequestLatencyMillis_max` and `openBlockRequestLatencyMillis_50thPercentile`
Second, the values of the metrics themselves have changed, to expose milliseconds instead of nanoseconds. Note that this does not affect metrics such as `openBlockRequestLatencyMillis_count` or `openBlockRequestLatencyMillis_rate1`, only the `Snapshot`-related metrics (`max`, `median`, percentiles, etc.). For the YARN case, these metrics were also introduced by #32388, and thus also have not yet been released. It was possible for the nanosecond values to be consumed by some other metrics reporter reading the Dropwizard metrics directly, but I'm not aware of any such usages.

### How was this patch tested?
Unit tests have been updated.

Closes #33116 from xkrogen/xkrogen-SPARK-35259-ess-fix-metric-unit-prefix.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: yi.wu <yi.wu@databricks.com>
(cherry picked from commit 70a15868fc)
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-07-24 21:27:56 +08:00
Ye Zhou 1907f0ac57 [SPARK-35546][SHUFFLE] Enable push-based shuffle when multiple app attempts are enabled and manage concurrent access to the state in a better way
### What changes were proposed in this pull request?
This is one of the patches for SPIP SPARK-30602 which is needed for push-based shuffle.

### Summary of the change:
When Executor registers with Shuffle Service, it will encode the merged shuffle dir created and also the application attemptId into the ShuffleManagerMeta into Json. Then in Shuffle Service, it will decode the Json string and get the correct merged shuffle dir and also the attemptId. If the registration comes from a newer attempt, the merged shuffle information will be updated to store the information from the newer attempt.

This PR also refactored the management of the merged shuffle information to avoid concurrency issues.
### 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.

Closes #33078 from zhouyejoe/SPARK-35546.

Authored-by: Ye Zhou <yezhou@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit c77acf0bbc)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-07-20 00:04:16 -05: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