Commit graph

125 commits

Author SHA1 Message Date
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
Venkata krishnan Sowrirajan 38dd42a50c [SPARK-36332][SHUFFLE] Cleanup RemoteBlockPushResolver log messages
### What changes were proposed in this pull request?
Cleanup `RemoteBlockPushResolver` log messages by using `AppShufflePartitionInfo#toString()` to avoid duplications. Currently this is based off of https://github.com/apache/spark/pull/33034 will remove those changes once it is merged and remove the WIP at that time.

### Why are the changes needed?
Minor cleanup to make code more readable.

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

### How was this patch tested?
No tests, just changing log messages

Closes #33561 from venkata91/SPARK-36332.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
(cherry picked from commit ab897109a3)
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-08-10 09:54:54 +08:00
Venkata krishnan Sowrirajan 0a98e51b84 [SPARK-32923][FOLLOW-UP] Clean up older shuffleMergeId shuffle files when finalize request for higher shuffleMergeId is received
### What changes were proposed in this pull request?

Clean up older shuffleMergeId shuffle files when finalize request for higher shuffleMergeId is received when no blocks pushed for the corresponding shuffleMergeId. This is identified as part of https://github.com/apache/spark/pull/33034#discussion_r680610872.

### Why are the changes needed?

Without this change, older shuffleMergeId files won't be cleaned up properly.

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

No

### How was this patch tested?

Added changes to existing unit test to address this case.

Closes #33605 from venkata91/SPARK-32923-follow-on.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit d8169493b6)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-04 03:30:56 -05:00
yi.wu df43300227 [SPARK-36206][CORE] Support shuffle data corruption diagnosis via shuffle checksum
### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out https://github.com/apache/spark/pull/32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

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

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes #33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit a98d919da4)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-02 09:59:30 -05: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
Min Shen bbec381f5e [SPARK-36266][SHUFFLE] Rename classes in shuffle RPC used for block push operations
### What changes were proposed in this pull request?
This is a follow-up to #29855 according to the [comments](https://github.com/apache/spark/pull/29855/files#r505536514)
In this PR, the following changes are made:

1. A new `BlockPushingListener` interface is created specifically for block push. The existing `BlockFetchingListener` interface is left as is, since it might be used by external shuffle solutions. These 2 interfaces are unified under `BlockTransferListener` to enable code reuse.
2. `RetryingBlockFetcher`, `BlockFetchStarter`, and `RetryingBlockFetchListener` are renamed to `RetryingBlockTransferor`, `BlockTransferStarter`, and `RetryingBlockTransferListener` respectively. This makes their names more generic to be reused across both block fetch and push.
3. Comments in `OneForOneBlockPusher` are further clarified to better explain how we handle retries for block push.

### Why are the changes needed?
To make code cleaner without sacrificing backward compatibility.

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

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

Closes #33340 from Victsm/SPARK-32915-followup.

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 c4aa54ed4e)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-07-26 17:40:19 -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
Chandni Singh 96944ac17d [SPARK-36255][SHUFFLE][CORE] Stop pushing and retrying on FileNotFound exceptions
### What changes were proposed in this pull request?
Once the shuffle is cleaned up by the `ContextCleaner`, the shuffle files are deleted by the executors. In this case, the push of the shuffle data by the executors can throw `FileNotFoundException`s because the shuffle files are deleted. When this exception is thrown from the `shuffle-block-push-thread`, it causes the executor to exit. Both the `shuffle-block-push` threads and the netty event-loops will encounter `FileNotFoundException`s in this case.  The fix here stops these threads from pushing more blocks when they encounter `FileNotFoundException`. When the exception is from the `shuffle-block-push-thread`, it will get handled and logged as warning instead of failing the executor.

### Why are the changes needed?
This fixes the bug which causes executor to exits when they are instructed to clean up shuffle data.
Below is the stacktrace of this exception:
```
21/06/17 16:03:57 ERROR util.SparkUncaughtExceptionHandler: Uncaught exception in thread Thread[block-push-thread-1,5,main]
java.lang.Error: java.io.IOException: Error in opening FileSegmentManagedBuffer

{file=********/application_1619720975011_11057757/blockmgr-560cb4cf-9918-4ea7-a007-a16c5e3a35fe/0a/shuffle_1_690_0.data, offset=10640, length=190}
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1155)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: Error in opening FileSegmentManagedBuffer\{file=*******/application_1619720975011_11057757/blockmgr-560cb4cf-9918-4ea7-a007-a16c5e3a35fe/0a/shuffle_1_690_0.data, offset=10640, length=190}

at org.apache.spark.network.buffer.FileSegmentManagedBuffer.nioByteBuffer(FileSegmentManagedBuffer.java:89)
at org.apache.spark.shuffle.ShuffleWriter.sliceReqBufferIntoBlockBuffers(ShuffleWriter.scala:294)
at org.apache.spark.shuffle.ShuffleWriter.org$apache$spark$shuffle$ShuffleWriter$$sendRequest(ShuffleWriter.scala:270)
at org.apache.spark.shuffle.ShuffleWriter.org$apache$spark$shuffle$ShuffleWriter$$pushUpToMax(ShuffleWriter.scala:191)
at org.apache.spark.shuffle.ShuffleWriter$$anon$2$$anon$4.run(ShuffleWriter.scala:244)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
... 2 more
Caused by: java.io.FileNotFoundException: ******/application_1619720975011_11057757/blockmgr-560cb4cf-9918-4ea7-a007-a16c5e3a35fe/0a/shuffle_1_690_0.data (No such file or directory)
at java.io.RandomAccessFile.open0(Native Method)
at java.io.RandomAccessFile.open(RandomAccessFile.java:316)
at java.io.RandomAccessFile.<init>(RandomAccessFile.java:243)
at org.apache.spark.network.buffer.FileSegmentManagedBuffer.nioByteBuffer(FileSegmentManagedBuffer.java:62)
```

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

### How was this patch tested?
Added a unit to verify no more data is pushed when `FileNotFoundException` is encountered. Have also verified in our environment.

Closes #33477 from otterc/SPARK-36255.

Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
(cherry picked from commit 09e1c61272)
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-07-24 21:10:01 +08:00
Almog Tavor 3a0184d15d [SPARK-36273][SHUFFLE] Fix identical values comparison
This commit fixes the use of the "o.appAttemptId" variable instead of the mistaken "appAttemptId" variable. The current situation is a comparison of identical values. Jira issue report SPARK-36273.

### What changes were proposed in this pull request?
This is a patch for SPARK-35546 which is needed for push-based shuffle.

### Why are the changes needed?
A very minor fix of adding the reference from the other "FinalizeShuffleMerge".

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

### How was this patch tested?
No unit tests were added. It's a pretty logical change.

Closes #33493 from almogtavor/patch-1.

Authored-by: Almog Tavor <70065337+almogtavor@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit 530c8addbb)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-07-23 12:46:42 -05: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
Erik Krogen 3255511d52 [SPARK-35258][SHUFFLE][YARN] Add new metrics to ExternalShuffleService for better monitoring
### What changes were proposed in this pull request?
This adds two new additional metrics to `ExternalBlockHandler`:
- `blockTransferRate` -- for indicating the rate of transferring blocks, vs. the data within them
- `blockTransferAvgSize_1min` -- a 1-minute trailing average of block sizes transferred by the ESS

Additionally, this enhances `YarnShuffleServiceMetrics` to expose the histogram/`Snapshot` information from `Timer` metrics within `ExternalBlockHandler`.

### Why are the changes needed?
Currently `ExternalBlockHandler` exposes some useful metrics, but is lacking around metrics for the rate of block transfers. We have `blockTransferRateBytes` to tell us the rate of _bytes_, but no metric to tell us the rate of _blocks_, which is especially relevant when running the ESS on HDDs that are sensitive to random reads. Many small block transfers can have a negative impact on performance, but won't show up as a spike in `blockTransferRateBytes` since the sizes are small. Thus the new metrics to show information around average block size and block transfer rate are very useful to monitor the health/performance of the ESS, especially when running on HDDs.

For the `YarnShuffleServiceMetrics`, currently the three `Timer` metrics exposed by `ExternalBlockHandler` are being underutilized in a YARN-based environment -- they are basically treated as a `Meter`, only exposing rate-based information, when the metrics themselves are collected detailed histograms of timing information. We should expose this information for better observability.

### Does this PR introduce _any_ user-facing change?
Yes, there are two entirely new metrics for the ESS, as documented in `monitoring.md`. Additionally in a YARN environment, `Timer` metrics exposed by the ESS will include more rich timing information.

### How was this patch tested?
New unit tests are added to verify that new metrics are showing up as expected.

We have been running this patch internally for approx. 1 year and have found it to be useful for monitoring the health of ESS and diagnosing performance issues.

Closes #32388 from xkrogen/xkrogen-SPARK-35258-ess-new-metrics.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-06-28 02:36:17 -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
Ye Zhou a97885bb2c [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data
### 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 changes:
Executor will create the merge directories under the application temp directory provided by YARN. The access control of the folder will be set to 770, where Shuffle Service can create merged shuffle files and write merge shuffle data in to those files.

Serve the merged shuffle blocks fetch request, read the merged shuffle blocks.

### 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 #32007 from zhouyejoe/SPARK-33350.

Lead-authored-by: Ye Zhou <yezhou@linkedin.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-06-10 16:57:46 -05:00
weixiuli 4869e437da [SPARK-35424][SHUFFLE] Remove some useless code in the ExternalBlockHandler
### What changes were proposed in this pull request?

Remove some useless code in the ExternalBlockHandler.

### Why are the changes needed?
There is some useless code in the ExternalBlockHandler, so we may remove it.

### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?

Exist unittests.

Closes #32571 from weixiuli/SPARK-35424.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-20 19:03:14 +09:00
Kousuke Saruta b4348b7e56 [SPARK-35420][BUILD] Replace the usage of toStringHelper with ToStringBuilder
### What changes were proposed in this pull request?

This PR replaces `toStringHelper`, an API which breaks in Guava 27.

### Why are the changes needed?

SPARK-30272 (#26911) removed usages which breaks in Guava 27 but `toStringHelper` is instroduced again.

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

No.

### How was this patch tested?

Build successfully finished with the following command.
```
build/sbt -Dguava.version=27.0-jre -Phive -Phive-thriftserver -Pyarn -Pmesos -Pkubernetes -Phadoop-cloud -Pdocker-integration-tests -Pkubernetes-integration-tests -Pkinesis-asl -Pspark-ganglia-lgpl package
```

Closes #32567 from sarutak/remove-old-guava-usage.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-17 21:46:35 +09:00
Chandni Singh 6d88212f79 [SPARK-34840][SHUFFLE] Fixes cases of corruption in merged shuffle …
### What changes were proposed in this pull request?
This PR fixes bugs that causes corruption of push-merged blocks when a client terminates while pushing block. `RemoteBlockPushResolver` was introduced in #30062 (SPARK-32916).

There are 2 scenarios where the merged blocks get corrupted:
1. `StreamCallback.onFailure()` is called more than once. Initially we assumed that the onFailure callback will be called just once per stream. However, we observed that this is called twice when a client connection is reset. When the client connection is reset then there are 2 events that get triggered in this order.
 - `exceptionCaught`. This event is propagated to `StreamInterceptor`. `StreamInterceptor.exceptionCaught()` invokes `callback.onFailure(streamId, cause)`. This is the first time StreamCallback.onFailure() will be invoked.
 - `channelInactive`. Since the channel closes, the `channelInactive` event gets triggered which again is propagated to `StreamInterceptor`. `StreamInterceptor.channelInactive()` invokes `callback.onFailure(streamId, new ClosedChannelException())`. This is the second time  StreamCallback.onFailure() will be invoked.

2. The flag `isWriting` is set prematurely to true. This introduces an edge case where a stream that is trying to merge a duplicate block (created because of a speculative task) may interfere with an active stream if the duplicate stream fails.

Also adding additional changes that improve the code.

1.  Using positional writes all the time because this simplifies the code and with microbenchmarking haven't seen any performance impact.
2. Additional minor changes suggested by mridulm during an internal review.

### Why are the changes needed?
These are bug fixes and simplify the code.

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

### How was this patch tested?
Added unit tests. I have also tested these changes in Linkedin's internal fork on a cluster.

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

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

Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-03-25 12:47:46 -05:00
yuhaiyang 4e43819611 [SPARK-34534] Fix blockIds order when use FetchShuffleBlocks to fetch blocks
### What changes were proposed in this pull request?

Fix a problems which can lead to data correctness after part blocks retry in `OneForOneBlockFetcher` when use `FetchShuffleBlocks` .

### Why are the changes needed?
This is a data correctness bug, It's is no problems when use old protocol to send `OpenBlocks` before fetch chunks in `OneForOneBlockFetcher`;
In latest branch, `OpenBlocks`  has been replaced to `FetchShuffleBlocks`. Howerver, `FetchShuffleBlocks` read shuffle blocks order is not the same as `blockIds` in `OneForOneBlockFetcher`; the `blockIds` is used to match blockId with shuffle data with index, now it is out of order;
It will lead to read wrong block chunk when some blocks fetch failed in `OneForOneBlockFetcher`, it will retry the rest of the blocks in `blockIds`  based on the `blockIds`'s order.

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

### How was this patch tested?

Closes #31643 from seayoun/yuhaiyang_fix_use_FetchShuffleBlocks_order.

Lead-authored-by: yuhaiyang <yuhaiyang@yuhaiyangs-MacBook-Pro.local>
Co-authored-by: yuhaiyang <yuhaiyang@172.19.25.126>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 12:25:38 +08: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
Ye Zhou 1bd897cbc4 [SPARK-32918][SHUFFLE] RPC implementation to support control plane coordination for push-based shuffle
### 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 changes:
This PR introduces a new RPC to be called within Driver. When the expected shuffle push wait time reaches, Driver will call this RPC to facilitate coordination of shuffle map/reduce stages and notify external shuffle services to finalize shuffle block merge for a given shuffle. Shuffle services also respond back the metadata about a merged shuffle partition back to the caller.

### 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?
This code snippets won't be called by any existing code and will be tested after the coordinated driver changes gets merged in SPARK-32920.

Lead-authored-by: Min Shen mshenlinkedin.com

Closes #30163 from zhouyejoe/SPARK-32918.

Lead-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-23 15:16:20 -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
Bo Yang 1299c8a81d [SPARK-33037][SHUFFLE] Remove knownManagers to support user's custom shuffle manager plugin
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

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

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

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

Closes #29916 from boy-uber/knownManagers.

Lead-authored-by: Bo Yang <boy@uber.com>
Co-authored-by: Bo Yang <boy-uber@users.noreply.github.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-10-02 20:26:46 -07:00
yi.wu e6fec33f18 [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled
### What changes were proposed in this pull request?

This PR adds support to read host-local shuffle data from disk directly when external shuffle service is disabled.

Similar to #25299, we first try to get local disk directories for the shuffle data, which is located at the same host with the current executor. The only difference is, in #25299, it gets the directories from the external shuffle service while in this PR, it gets the directory from the executors.

To implement the feature, this PR extends the `HostLocalDirManager ` for both `ExternalBlockStoreClient` and `NettyBlockTransferService`. Also, this PR adds `getHostLocalDirs` for `NettyBlockTransferService` as `ExternalBlockStoreClient` does, in order to send the get-dir-request to the corresponding executor. And this PR resued the request message`GetLocalDirsForExecutors` for simple.

### Why are the changes needed?

After SPARK-27651 / #25299, Spark can read host-local shuffle data directly from disk when external shuffle service is enabled. To extend the future, we can also support it when the external shuffle service is disabled.

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

Yes. Before this PR, to use the host-local shuffle reading feature, users should not only enable `spark.shuffle.readHostLocalDisk` but also `spark.shuffle.service.enabled`. After this PR, enable `spark.shuffle.readHostLocalDisk` should be enough, and external shuffle service is no longer a pre-requirement.

### How was this patch tested?

Added test and tested manually.

Closes #28911 from Ngone51/support_node_local_shuffle.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-02 13:03:44 -07:00
“attilapiros” 79b4dea1b0 [SPARK-32663][CORE] Avoid individual closing of pooled TransportClients (which must be closed through the pool)
### What changes were proposed in this pull request?

Removing the individual `close` method calls on the pooled `TransportClient` instances.
The pooled clients should be only closed via `TransportClientFactory#close()`.

### Why are the changes needed?

Reusing a closed `TransportClient` leads to the exception `java.nio.channels.ClosedChannelException`.

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

No.

### How was this patch tested?

This is a trivial case which is not tested by specific test.

Closes #29492 from attilapiros/SPARK-32663.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2020-08-21 01:02:33 -05:00
“attilapiros” 1b3fc9a111 [SPARK-32149][SHUFFLE] Improve file path name normalisation at block resolution within the external shuffle service
### What changes were proposed in this pull request?

Improving file path name normalisation by removing the approximate transformation from Spark and using the path normalization from the JDK.

### Why are the changes needed?

In the external shuffle service during the block resolution the file paths (for disk persisted RDD and for shuffle blocks) are normalized by a custom Spark code which uses an OS dependent regexp. This is a redundant code of the package-private JDK counterpart. As the code not a perfect match even it could happen one method results in a bit different (but semantically equal) path.

The reason of this redundant transformation is the interning of the normalized path to save some heap here which is only possible if both transformations results in the same string.

Checking the JDK code I believe there is a better solution which is perfect match for the JDK code as it uses that package private method. Moreover based on some benchmarking even this new method seams to be more performant too.

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

No

### How was this patch tested?

As we are reusing the JDK code for normalisation no test is needed. Even the existing test can be removed.

But in a separate branch I have created a benchmark where the performance of the old and the new solution can be compared. It shows the new method is about 7-10 times better than old one.

Closes #28967 from attilapiros/SPARK-32149.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-11 22:55:26 +09:00
pancheng 7fda184f0f [SPARK-32121][SHUFFLE] Support Windows OS in ExecutorDiskUtils
### What changes were proposed in this pull request?
Correct file seprate use in `ExecutorDiskUtils.createNormalizedInternedPathname` on Windows

### Why are the changes needed?
`ExternalShuffleBlockResolverSuite` failed on Windows, see detail at:
https://issues.apache.org/jira/browse/SPARK-32121

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

### How was this patch tested?
The existed test suite.

Closes #28940 from pan3793/SPARK-32121.

Lead-authored-by: pancheng <379377944@qq.com>
Co-authored-by: chengpan <cheng.pan@idiaoyan.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-02 19:21:11 +09:00
tianlzhang dad61ed465
[SPARK-31646][SHUFFLE] Remove unused registeredConnections counter from ShuffleMetrics
### What changes were proposed in this pull request?
Remove unused `registeredConnections` counter from `ExternalBlockHandler#ShuffleMetrics`

This was added by SPARK-25642 at 3.0.0
- 8dd29fe36b

### Why are the changes needed?
It's `registeredConnections` counter created in `TransportContext` that's really counting the numbers and it's misleading for people who want to add new metrics like `registeredConnections`.

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

### How was this patch tested?
Add UTs to ensure all expected metrics are registered for `ExternalShuffleService` and `YarnShuffleService`

Closes #28457 from manuzhang/spark-31611-pre.

Lead-authored-by: tianlzhang <tianlzhang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-07 15:22:13 -07:00
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
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
Liang-Chi Hsieh 0042ad575a [SPARK-30290][CORE] Count for merged block when fetching continuous blocks in batch
### What changes were proposed in this pull request?

We added shuffle block fetch optimization in SPARK-9853. In ShuffleBlockFetcherIterator, we merge single blocks into batch blocks. During merging, we should count merged blocks for `maxBlocksInFlightPerAddress`, not original single blocks.

### Why are the changes needed?

If `maxBlocksInFlightPerAddress` is specified, like set it to 1, it should mean one batch block, not one original single block. Otherwise, it will conflict with batch shuffle fetch.

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

No

### How was this patch tested?

Unit test.

Closes #26930 from viirya/respect-max-blocks-inflight.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 18:57:02 +08: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
“attilapiros” fd2bf55aba [SPARK-27651][CORE] Avoid the network when shuffle blocks are fetched from the same host
## What changes were proposed in this pull request?

Before this PR `ShuffleBlockFetcherIterator` was partitioning the block fetches into two distinct sets: local reads and remote fetches. Within this PR (when the feature is enabled by "spark.shuffle.readHostLocalDisk.enabled") a new category is introduced: host-local reads. They are shuffle block fetches where although the block manager is different they are running on the same host along with the requester.

Moreover to get the local directories of the other executors/block managers a new RPC message is introduced `GetLocalDirs` which is sent the the block manager master where it is answered as `BlockManagerLocalDirs`. In `BlockManagerMasterEndpoint` for answering this request the `localDirs` is extracted from the `BlockManagerInfo` and stored separately in a hash map called `executorIdLocalDirs`. Because the earlier used `blockManagerInfo` contains data for the alive block managers (see `org.apache.spark.storage.BlockManagerMasterEndpoint#removeBlockManager`).

Now `executorIdLocalDirs` knows all the local dirs up to the application start (like the external shuffle service does) so in case of an RDD recalculation both host-local shuffle blocks and disk persisted RDD blocks on the same host can be served by reading the files behind the blocks directly.

## How was this patch tested?

### Unit tests

`ExternalShuffleServiceSuite`:
- "SPARK-27651: host local disk reading avoids external shuffle service on the same node"

`ShuffleBlockFetcherIteratorSuite`:
- "successful 3 local reads + 4 host local reads + 2 remote reads"

And with extending existing suites where shuffle metrics was tested.

### Manual tests

Running Spark on YARN in a 4 nodes cluster with 6 executors and having 12 shuffle blocks.

```
$ grep host-local experiment.log
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_2_1, shuffle_0_6_1
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 38 ms
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_0_0, shuffle_0_8_0
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 35 ms
```

Closes #25299 from attilapiros/SPARK-27651.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-26 11:02:25 -08:00
Sean Owen 1febd373ea [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests
### What changes were proposed in this pull request?

Use JUnit assertions in tests uniformly, not JVM assert() statements.

### Why are the changes needed?

assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.

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

No. The assertion logic should be identical.

### How was this patch tested?

Existing tests.

Closes #26581 from srowen/assertToJUnit.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 14:04:15 -06: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 239ee3f561 [SPARK-9853][CORE] Optimize shuffle fetch of continuous partition IDs
This PR takes over #19788. After we split the shuffle fetch protocol from `OpenBlock` in #24565, this optimization can be extended in the new shuffle protocol. Credit to yucai, closes #19788.

### What changes were proposed in this pull request?
This PR adds the support for continuous shuffle block fetching in batch:

- Shuffle client changes:
    - Add new feature tag `spark.shuffle.fetchContinuousBlocksInBatch`, implement the decision logic in `BlockStoreShuffleReader`.
    - Merge the continuous shuffle block ids in batch if needed in ShuffleBlockFetcherIterator.
- Shuffle server changes:
    - Add support in `ExternalBlockHandler` for the external shuffle service side.
    - Make `ShuffleBlockResolver.getBlockData` accept getting block data by range.
- Protocol changes:
    - Add new block id type `ShuffleBlockBatchId` represent continuous shuffle block ids.
    - Extend `FetchShuffleBlocks` and `OneForOneBlockFetcher`.
    - After the new shuffle fetch protocol completed in #24565, the backward compatibility for external shuffle service can be controlled by `spark.shuffle.useOldFetchProtocol`.

### Why are the changes needed?
In adaptive execution, one reducer may fetch multiple continuous shuffle blocks from one map output file. However, as the original approach, each reducer needs to fetch those 10 reducer blocks one by one. This way needs many IO and impacts performance. This PR is to support fetching those continuous shuffle blocks in one IO (batch way). See below example:

The shuffle block is stored like below:
![image](https://user-images.githubusercontent.com/2989575/51654634-c37fbd80-1fd3-11e9-935e-5652863676c3.png)
The ShuffleId format is s"shuffle_$shuffleId_$mapId_$reduceId", referring to BlockId.scala.

In adaptive execution, one reducer may want to read output for reducer 5 to 14, whose block Ids are from shuffle_0_x_5 to shuffle_0_x_14.
Before this PR, Spark needs 10 disk IOs + 10 network IOs for each output file.
After this PR, Spark only needs 1 disk IO and 1 network IO. This way can reduce IO dramatically.

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

### How was this patch tested?
Add new UT.
Integrate test with setting `spark.sql.adaptive.enabled=true`.

Closes #26040 from xuanyuanking/SPARK-9853.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-17 14:47:56 +08:00
Liang-Chi Hsieh 93e71e60e6 [SPARK-29469][SHUFFLE] Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed
### What changes were proposed in this pull request?

When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE. This proposes to skip retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed.

### Why are the changes needed?

When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE:

```
2019-10-14 20:06:16 ERROR RetryingBlockFetcher:143 - Exception while beginning fetch of 2 outstanding blocks (after 3 retries)
java.lang.NullPointerException
at org.apache.spark.network.shuffle.ExternalShuffleClient.lambda$fetchBlocks$0(ExternalShuffleClient.java:100)
at org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141)
at org.apache.spark.network.shuffle.RetryingBlockFetcher.lambda$initiateRetry$0(RetryingBlockFetcher.java:169)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
```

It was happened after BlockManager and ExternalBlockStoreClient was closed due to previous errors. In this cases, RetryingBlockFetcher does not need to retry. This NPE is harmless for job execution, but is a source of misleading when looking at log. Especially for end-users.

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

No

### How was this patch tested?

Existing tests.

Closes #26115 from viirya/SPARK-29469.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-16 13:11:07 +08: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
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
Yuanjian Li db39f45baf [SPARK-28593][CORE] Rename ShuffleClient to BlockStoreClient which more close to its usage
## What changes were proposed in this pull request?

After SPARK-27677, the shuffle client not only handles the shuffle block but also responsible for local persist RDD blocks. For better code scalability and precise semantics(as the [discussion](https://github.com/apache/spark/pull/24892#discussion_r300173331)), here we did several changes:

- Rename ShuffleClient to BlockStoreClient.
- Correspondingly rename the ExternalShuffleClient to ExternalBlockStoreClient, also change the server-side class from ExternalShuffleBlockHandler to ExternalBlockHandler.
- Move MesosExternalBlockStoreClient to Mesos package.

Note, we still keep the name of BlockTransferService, because the `Service` contains both client and server, also the name of BlockTransferService is not referencing shuffle client only.

## How was this patch tested?

Existing UT.

Closes #25327 from xuanyuanking/SPARK-28593.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-05 14:54:45 +08: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
“attilapiros” b71c130fc6 [SPARK-27622][CORE] Avoiding the network when block manager fetches disk persisted RDD blocks from the same host
## What changes were proposed in this pull request?

Before this PR during fetching a disk persisted RDD block the network was always used to get the requested block content even when both the source and fetcher executor was running on the same host.

The idea to access another executor local disk files by directly reading the disk comes from the external shuffle service where the local dirs are stored for each executor (block manager).

To make this possible the following changes are done:
- `RegisterBlockManager` message is extended with the `localDirs` which is stored by the block manager master for each block manager as a new property of the `BlockManagerInfo`
- `GetLocationsAndStatus` is extended with the requester host
- `BlockLocationsAndStatus` (the reply for `GetLocationsAndStatus` message) is extended with the an option of local directories, which is filled with a local directories of a same host executor (if there is any, otherwise None is used). This is where the block content can be read from.

Shuffle blocks are out of scope of this PR: there will be a separate PR opened for that (for another Jira issue).

## How was this patch tested?

With a new unit test in `BlockManagerSuite`. See the the test prefixed by "SPARK-27622: avoid the network when block requested from same host".

Closes #24554 from attilapiros/SPARK-27622.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-25 07:35:44 -07:00