### What changes were proposed in this pull request?
MapperAttempts for a shuffle replies the `MAP_ENDED` when mapper has already been ended for receving push data or push merged data from speculative task.
Follow up #1591.
### Why are the changes needed?
When mapper has already been ended for receving push data or push merged data from speculative task, `PushDataHandler` should trigger MapEnd instead of StageEnd for worker. Meanwhile, the `ShuffleClientImpl` should handle `STAGE_ENDED` as MapEnd, otherwise causes that other tasks of the stage could not send shuffle data for data lost.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Internal test.
Closes#2190 from SteNicholas/CELEBORN-678.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
1. Fix MissingOverride, DefaultCharset, UnnecessaryParentheses Rule
2. Exclude generated sources, FutureReturnValueIgnored, TypeParameterUnusedInFormals, UnusedVariable
### Why are the changes needed?
```
./build/make-distribution.sh --release
```
We get a lot of WARNINGs.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#2177 from cxzl25/error_prone_patch.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
Update log level of CommitFiles success for `CommitHandler` from error to info.
### Why are the changes needed?
The log level of sending CommitFiles success for `CommitHandler` should not be error.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2174 from SteNicholas/commit-files-log.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
1. To support io encryption for spark.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and manually test on a cluster.
Closes#2135 from FMX/B1150.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
To avoid NPE in `val future = workerInfo.endpoint.ask[DestroyWorkerSlotsResponse](destroy)`
### Why are the changes needed?
ditto
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test
Closes#2166 from waitinfuture/1181.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
I'm testing main branch and encountered the following scenario.
I run `sbin/stop-worker.sh` near simultaneously on 3 out of 6 workers, and I'm expecting the 3 workers
will soon shutdown because I enabled graceful shutdown. However, only the first worker I stopped
shutdown in 15s as expected, the other two won't shutdown until shutdown timeout.
After digging into it, I found `LifecycleManager#reserveSlotsWithRetry` will reserve for the same location
twice:
1. At T1, only worker1 shutdown, pushes receive HARD_SPLIT and goes to revive
2. At T2, LifecycleManager handles revive requests in batch, and try to reallocate the locs to other workers
3. At T3, reserve to worker3 succeeds because it's not shutdown yet, but reserve to worker2 fails because it's shutdown
4. At T4, LifecycleManager will re-allocate the failed slots to other workers except worker1 and worker2. However, at this time Worker3 is also shutdown, so it fails to reserve on worker3
5. At T5, it re-allocates slots that failed to worker3. However, `getFailedPartitionLocations` will return slots allocated to worker3 in step 3, and increment the epoch to 2. At this time, worker3 has slots of epoch 1, but they will never to pushed to because newer epoch 3 is generated at the same time
6. Since the epoch 2 locs in worker3 will never be pushed to, it will never get a chance to return HARD_SPLIT, as a result it can't fast shutdown untile timeout.
This PR fixes this by destroying failed to be reserved slots in the process of `reserveSlotsWithRetry`
### Why are the changes needed?
ditto
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test.
Before:

After:

Closes#2163 from waitinfuture/1178.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title.
### Why are the changes needed?
As title.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Passes UTs.
Closes#2162 from waitinfuture/1175-2.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
One user reported that LifecycleManager's parmap can create huge number of threads and causes OOM.

There are four places where parmap is called:
1. When LifecycleManager commits files
2. When LifecycleManager reserves slots
3. When LifecycleManager setup connection to workers
4. When LifecycleManager call destroy slots
This PR fixes the fourth one. To be more detail, this PR eliminates `parmap` when destroying slots, and also replaces `askSync` with `ask`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test and GA.
Closes#2156 from waitinfuture/1167.
Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: cxzl25 <cxzl25@users.noreply.github.com>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
as title
### Why are the changes needed?
as title
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Passes GA
Closes#2159 from waitinfuture/1171.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
One user reported that LifecycleManager's parmap can create huge number of threads and causes OOM.

There are four places where parmap is called:
1. When LifecycleManager commits files
2. When LifecycleManager reserves slots
3. When LifecycleManager setup connection to workers
4. When StorageManager calls close
This PR fixes the third one. To be more detail, this PR eliminates `parmap` when setup connection to workers, and also replaces `askSync` with `ask`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test and GA.
Closes#2154 from waitinfuture/1166.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
One user reported that LifecycleManager's parmap can create huge number of threads and causes OOM.

There are four places where parmap is called:
1. When LifecycleManager commits files
2. When LifecycleManager reserves slots
3. When LifecycleManager setup connection to workers
4. When StorageManager calls close
This PR fixes the second one. To be more detail, this PR eliminates `parmap` when reserving slots, and also replaces `askSync` with `ask`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test and GA.
Closes#2152 from waitinfuture/1165-1.
Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
One user reported that LifecycleManager's parmap can create huge number of threads and causes OOM.

There are four places where parmap is called:
1. When LifecycleManager commits files
2. When LifecycleManager reserves slots
3. When LifecycleManager setup connection to workers
4. When StorageManager calls close
This PR fixes the first one. To be more detail, this PR eliminates `parmap` when doing committing files, and also replaces `askSync` with `ask`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test and GA.
Closes#2145 from waitinfuture/1160.
Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
When request slots, filter workers excluded by application
### Why are the changes needed?
If worker alive but can not service, register shuffle will remove the worker from application client exclude list and next shuffle may reserve slots on this worker,this will cause application revive unexpectly
### Does this PR introduce _any_ user-facing change?
Yes, request slots will filter workers excluded by application
### How was this patch tested?
UT,
Closes#2131 from wangshengjie123/fix-request-slots-blacklist.
Authored-by: wangshengjie <wangshengjie3@xiaomi.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
In [celeborn-955](https://github.com/apache/incubator-celeborn/pull/1924), GetShuffleId RPC was introduced to generate a celeborn shuffle id from app shuffle id to support spark stage rerun
GetShuffleId RPC assumes that Shuffle Write operation always happens before Shuffle Read operation, but this is not true for empty shuffle data in celeborn, which causes GetShuffleId RPC to throw NPE and fail the Job
This PR fixes this bug
### Why are the changes needed?
to avoid spark job failure with empty shuffle data
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
a new test case is included for empty shuffle data
Closes#2136 from ErikFang/fix-GetShuffleId-RPC-NPE-for-empty-shuffle.
Lead-authored-by: Erik.fang <fmerik@gmail.com>
Co-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
The `clientPushBufferMaxSize` config is also used by `CelebornInputStreamImpl`, it's a config about push side and should not be used by fetch side. This pr introduces a fetch config to replace it.
### Why are the changes needed?
As above
### Does this PR introduce _any_ user-facing change?
Yes, a new config `celeborn.client.fetch.buffer.size` is introduced.
### How was this patch tested?
Pass CI
Closes#2118 from exmy/celeborn-1145.
Authored-by: exmy <xumovens@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
As Title
### Why are the changes needed?
As Title
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
PASS GA
Closes#2114 from jiaoqingbo/1142.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
Currently, Celeborn uses replication to handle shuffle data lost for celeborn shuffle reader, this PR implements an alternative solution by Spark stage resubmission.
Design doc:
https://docs.google.com/document/d/1dkG6fww3g99VAb1wkphNlUES_MpngVPNg8601chmVp8/edit
### Why are the changes needed?
Spark stage resubmission uses less resources compared with replication, and some Celeborn users are also asking for it
### Does this PR introduce _any_ user-facing change?
a new config celeborn.client.fetch.throwsFetchFailure is introduced to enable this feature
### How was this patch tested?
two UTs are attached, and we also tested it in Ant Group's Dev spark cluster
Closes#1924 from ErikFang/Re-run-Spark-Stage-for-Celeborn-Shuffle-Fetch-Failure.
Lead-authored-by: Erik.fang <fmerik@gmail.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
As Title
### Why are the changes needed?
As Title
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
PASS GA
Closes#2113 from jiaoqingbo/1140.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
As Title
As Title
NO
PASS GA
Closes#2111 from jiaoqingbo/1138.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Add lastException to CelebornIOException when createReaderWithRetry meet error
### Why are the changes needed?
Now we should to find the detail executor to dedicate the detail error msg
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
Closes#2103 from wxplovecc/easy-to-dedicate-error.
Authored-by: 吴祥平 <wxp4532@ly.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
When I kill -9 a Worker process, Master will not exclude the worker until heartbeat timeout.
During this time, Master will still allocate slots on this Worker, causing NPE when register shuffle
```
Caused by: java.lang.NullPointerException
at org.apache.celeborn.client.LifecycleManager.requestWorkerReserveSlots(LifecycleManager.scala:1246) ~[celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar:?]
at org.apache.celeborn.client.LifecycleManager.$anonfun$reserveSlots$2(LifecycleManager.scala:864) ~[celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar:?]
at org.apache.celeborn.common.util.ThreadUtils$.$anonfun$parmap$2(ThreadUtils.scala:301) ~[celeborn-client-spark-3-shaded_2.12-0.4.0-SNAPSHOT.jar:?]
at scala.concurrent.Future$.$anonfun$apply$1(Future.scala:659) ~[scala-library-2.12.15.jar:?]
at scala.util.Success.$anonfun$map$1(Try.scala:255) ~[scala-library-2.12.15.jar:?]
at scala.util.Success.map(Try.scala:213) ~[scala-library-2.12.15.jar:?]
at scala.concurrent.Future.$anonfun$map$1(Future.scala:292) ~[scala-library-2.12.15.jar:?]
at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33) ~[scala-library-2.12.15.jar:?]
at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33) ~[scala-library-2.12.15.jar:?]
at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64) ~[scala-library-2.12.15.jar:?]
at java.util.concurrent.ForkJoinTask$RunnableExecuteAction.exec(ForkJoinTask.java:1402) ~[?:1.8.0_372]
at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_372]
at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) ~[?:1.8.0_372]
at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) ~[?:1.8.0_372]
at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) ~[?:1.8.0_372]
```
### Why are the changes needed?
ditto
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test and passes GA
Closes#2104 from waitinfuture/1130.
Authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
ConcurrentHashMap.contains main containsValue ,not containsKey. In the current codebase, there is a misuse of the contains method in the ConcurrentHashMap class.
### Why are the changes needed?
ConcurrentHashMap.contains misuse
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No
Closes#2102 from lyy-pineapple/hashMap.
Authored-by: liangyongyuan <liangyongyuan@xiaomi.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Exclude workers of shuffle manager remove worker of connect exception primary or replica.
### Why are the changes needed?
Exclude workers of shuffle manager should not always remove worker of connect exception replica.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2091 from SteNicholas/CELEBORN-1124.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
`ShuffleClientImpl` closes `batchReviveRequestScheduler` of `ReviveManager`.
### Why are the changes needed?
After shuffle client is closed, `ReviveManager` still schedules invoker to `ShuffleClientImpl#reviveBatch`, which causes the `NullPointerException`. Therefore, `ShuffleClientImpl` should close `batchReviveRequestScheduler` of `ReviveManager` to avoid `NullPointerException`.
```
23/11/08 18:09:25,819 [batch-revive-scheduler] ERROR ShuffleClientImpl: Exception raised while reviving for shuffle 0 partitionIds 1988, epochs 0,.
java.lang.NullPointerException
at org.apache.celeborn.client.ShuffleClientImpl.reviveBatch(ShuffleClientImpl.java:705)
at org.apache.celeborn.client.ReviveManager.lambda$new$1(ReviveManager.java:94)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
23/11/08 18:09:25,844 [celeborn-retry-sender-6] ERROR ShuffleClientImpl: Push data to xx.xx.xx.xx:9092 failed for shuffle 0 map 216 attempt 0 partition 1988 batch 2623, remain revive times 4.
org.apache.celeborn.common.exception.CelebornIOException: PUSH_DATA_CONNECTION_EXCEPTION_PRIMARY then revive but REVIVE_FAILED, revive status 12(REVIVE_FAILED), old location: PartitionLocation[
id-epoch:1988-0
host-rpcPort-pushPort-fetchPort-replicatePort:xx.xx.xx.xx-9091-9092-9093-9094
mode:PRIMARY
peer:(empty)
storage hint:StorageInfo{type=MEMORY, mountPoint='/tmp/storage', finalResult=false, filePath=}
mapIdBitMap:null]
at org.apache.celeborn.client.ShuffleClientImpl.submitRetryPushData(ShuffleClientImpl.java:261)
at org.apache.celeborn.client.ShuffleClientImpl.access$600(ShuffleClientImpl.java:62)
at org.apache.celeborn.client.ShuffleClientImpl$3.lambda$onFailure$1(ShuffleClientImpl.java:1045)
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 java.lang.Thread.run(Thread.java:748)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2084 from SteNicholas/CELEBORN-1120.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
Unregister application to Celeborn master After Application stopped, then master will expire the related shuffle resource immediately, resulting in resource savings.
### Why are the changes needed?
Currently Celeborn master expires the related application shuffle resource only when application is being checked timeout,
this would greatly delay the release of resources, which is not conducive to saving resources.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
PASS GA
Closes#2075 from RexXiong/CELEBORN-1112.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Cache RegisterShuffleResponse to improve the processing speed of LifecycleManager
### Why are the changes needed?
During the processing of the registerShuffle request, constructing the RegisterShuffleResponse instance and serialization can indeed consume a significant amount of time. When there are a large number of registerShuffle requests that need to be processed by the LifecycleManager simultaneously, the response time of the LifecycleManager will be delayed. Therefore, caching is needed to improve the processing performance of the LifecycleManager.

### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2070 from kerwin-zk/issue-1109.
Authored-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Let the MR module compile successfully.
### Why are the changes needed?
#2000 added parameters in the `ShuffleClient#readPartition` method, resulting in MR module compilation failure.
MR CI is still missing.
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
local test
```bash
./build/make-distribution.sh -Pmr
```
Closes#2069 from cxzl25/CELEBORN-1048-FOLLOWUP.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
1.To support `celeborn.storage.activeTypes` in Client.
2.Master will ignore slots for "UNKNOWN_DISK".
### Why are the changes needed?
Enable client application to select storage types to use.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
GA and cluster.
Closes#2045 from FMX/B1081.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Support specifying the number of dispatcher threads for each role, especially shuffle client side. For shuffle client, there is only RpcEndpointVerifier endpoint which handles not many requests, one thread is enough. The rpc env of other roles has only two endpoints at most, using a shared event loop is reasonable. I am not sure if there is a need to add rpc requests to shuffle client. So add specific parameters to specify the dispatcher threads here.
And change the dispatcher thread pool name in order to distinguish it from spark's.
### Why are the changes needed?
Ditto
### Does this PR introduce _any_ user-facing change?
Yes, add params celeborn.\<role>.rpc.dispatcher.threads
### How was this patch tested?
Manual test and UT
Closes#2003 from onebox-li/my_dev.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
When retry push data and revive succeed in ShuffleClientImpl#submitRetryPushData, if new location is excluded, the callback's `lastest` location has not been updated when wrappedCallback.onFailure is called in ShuffleClientImpl#isPushTargetWorkerExcluded. Therefore there may be problems with subsequent revive.
### Why are the changes needed?
Ditto
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test.
Closes#2005 from onebox-li/improve-push-exclude.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
At present, from the log, We don't know which worker's request timed out.
```java
23/10/30 15:44:51,963 [CommitFiles-ForkJoinPool-162-worker-1] ERROR ReducePartitionCommitHandler: AskSync CommitFiles for 0 failed (attempt 1/4).
org.apache.celeborn.common.rpc.RpcTimeoutException: Futures timed out after [60000 milliseconds]. This timeout is controlled by celeborn.rpc.askTimeout
at org.apache.celeborn.common.rpc.RpcTimeout.org$apache$celeborn$common$rpc$RpcTimeout$$createRpcTimeoutException(RpcTimeout.scala:46)
at org.apache.celeborn.common.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:61)
at org.apache.celeborn.common.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:57)
at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38)
at org.apache.celeborn.common.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
at org.apache.celeborn.common.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:89)
at org.apache.celeborn.common.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:73)
at org.apache.celeborn.client.commit.CommitHandler.requestCommitFilesWithRetry(CommitHandler.scala:417)
at org.apache.celeborn.client.commit.CommitHandler.commitFiles(CommitHandler.scala:279)
at org.apache.celeborn.client.CommitManager$$anon$1$$anon$2.$anonfun$run$2(CommitManager.scala:151)
at org.apache.celeborn.client.CommitManager$$anon$1$$anon$2.$anonfun$run$2$adapted(CommitManager.scala:122)
at org.apache.celeborn.common.util.ThreadUtils$.$anonfun$parmap$2(ThreadUtils.scala:293)
at scala.concurrent.Future$.$anonfun$apply$1(Future.scala:659)
at scala.util.Success.$anonfun$map$1(Try.scala:255)
at scala.util.Success.map(Try.scala:213)
at scala.concurrent.Future.$anonfun$map$1(Future.scala:292)
at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33)
at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33)
at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64)
at java.util.concurrent.ForkJoinTask$RunnableExecuteAction.exec(ForkJoinTask.java:1402)
at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
Caused by: java.util.concurrent.TimeoutException: Futures timed out after [60000 milliseconds]
at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:259)
at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:263)
at org.apache.celeborn.common.util.ThreadUtils$.awaitResult(ThreadUtils.scala:225)
at org.apache.celeborn.common.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:74)
... 19 more
```
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2054 from cxzl25/CELEBORN-1098.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
For spark case, when stage outputPartitioning is satisfied and no longer needs shuffle exchange, there will be no shuffle write procedure, same goes for `RegisterShuffle`, currently the origin reduce stage will throw a NPE when LifeCycleManager `handleGetReducerFileGroup`.
```
ERROR [dispatcher-event-loop-11] Inbox: Ignoring error
java.lang.NullPointerException: null
at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.handleGetReducerFileGroup(ReducePartitionCommitHandler.scala:307)
at org.apache.celeborn.client.CommitManager.handleGetReducerFileGroup(CommitManager.scala:266)
at org.apache.celeborn.client.LifecycleManager.org$apache$celeborn$client$LifecycleManager$$handleGetReducerFileGroup(LifecycleManager.scala:556)
at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:298)
at org.apache.celeborn.common.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
at org.apache.celeborn.common.rpc.netty.Inbox.safelyCall(Inbox.scala:222)
at org.apache.celeborn.common.rpc.netty.Inbox.process(Inbox.scala:110)
at org.apache.celeborn.common.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:227)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
Reproduce example like:
`select count(*) as cnt from tableA;`
And tableA is empty.
So here fix code to adapt to this normal situation. For Flink client, just follows the old behavior.
### Why are the changes needed?
Fix the probable NPE.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Cluster test.
Closes#2056 from onebox-li/fix-empty-shuffle-npe.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`CelebornConf` adds `celeborn.client.shuffle.decompression.lz4.xxhash.instance` to configure fastest available `XXHashFactory` instance for checksum of `Lz4Decompressor`. Fix#2043.
### Why are the changes needed?
`Lz4Decompressor` creates the checksum with `XXHashFactory#fastestInstance`, which returns the fastest available `XXHashFactory` instance that uses nativeInstance at default. The fastest available `XXHashFactory` instance for checksum of `Lz4Decompressor` could be supported to configure instead of dependency on the class loader is the system class loader, which method is as follows:
```
/**
* Returns the fastest available {link XXHashFactory} instance. If the class
* loader is the system class loader and if the
* {link #nativeInstance() native instance} loads successfully, then the
* {link #nativeInstance() native instance} is returned, otherwise the
* {link #fastestJavaInstance() fastest Java instance} is returned.
* <p>
* Please read {link #nativeInstance() javadocs of nativeInstance()} before
* using this method.
*
* return the fastest available {link XXHashFactory} instance.
*/
public static XXHashFactory fastestInstance() {
if (Native.isLoaded()
|| Native.class.getClassLoader() == ClassLoader.getSystemClassLoader()) {
try {
return nativeInstance();
} catch (Throwable t) {
return fastestJavaInstance();
}
} else {
return fastestJavaInstance();
}
}
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `CelebornConfSuite`
- `ConfigurationSuite`
Closes#2050 from SteNicholas/CELEBORN-1095.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
### What changes were proposed in this pull request?
Use `scheduleWithFixedDelay` instead of `scheduleAtFixedRate` in thread pool of Celeborn Master and Worker.
### Why are the changes needed?
Follow up #1970.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Internal tests.
Closes#2048 from SteNicholas/CELEBORN-1032.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Remove unnecessary increment index of `Master#timeoutDeadWorkers`.
### Why are the changes needed?
Increment index of `Master#timeoutDeadWorkers` is unnecessary.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2027 from SteNicholas/timeout-dead-workers.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1994 from cxzl25/CELEBORN-1042.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
Adjust the local reader logs. Before, it will log local read stats in each stream clos whether it really contains local read.
And refine the CelebornInputStreamImpl#createReader code to be more clearer.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
Adjust local read logs.
### How was this patch tested?
Manual test.
Closes#1988 from onebox-li/local-dev.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`batchHandleChangePartitionExecutors` could not handle uncaught exception in `ChangePartitionRequest`, which causes that the uncaught exception of the thread could not get for troubleshooting. Thread factory should set `UncaughtExceptionHandler` to handle uncaught exception.
### Why are the changes needed?
Thread factory sets `UncaughtExceptionHandler` to handle uncaught exception in `ThreadUtils`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit tests.
Closes#1962 from SteNicholas/CELEBORN-1024.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
1. this is developer-friendly for debugging unit tests in IntelliJ IDEA, for example: Netty's memory leak reports are logged at the error level and won't cause unit tests to be marked as fatal.
```
23/10/09 09:57:26,422 ERROR [fetch-server-52-2] ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's garbage-collected. See https://netty.io/wiki/reference-counted-objects.html for more information.
Recent access records:
Created at:
io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:403)
io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188)
io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:179)
io.netty.buffer.AbstractByteBufAllocator.ioBuffer(AbstractByteBufAllocator.java:140)
io.netty.channel.DefaultMaxMessagesRecvByteBufAllocator$MaxMessageHandle.allocate(DefaultMaxMessagesRecvByteBufAllocator.java:120)
io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:150)
io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
java.lang.Thread.run(Thread.java:750)
```
2. this won't increase console output and affect the stability of CI.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GA
Closes#1958 from cfmcgrady/ut-console-log-level.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
The `ReusedExchange` operator has the potential to generate different types of fetch requests, including both non-range and range requests. Currently, an issue arises due to the synchronous deletion of the original file by the Celeborn worker upon completion of sorting. This issue leads to the failure of non-range requests following a range request for the same partition.
the snippets to reproduce this bug
```scala
val sparkConf = new SparkConf().setAppName("celeborn-test").setMaster("local[2]")
.set("spark.shuffle.manager", "org.apache.spark.shuffle.celeborn.SparkShuffleManager")
.set(s"spark.${CelebornConf.MASTER_ENDPOINTS.key}", masterInfo._1.rpcEnv.address.toString)
.set("spark.sql.autoBroadcastJoinThreshold", "-1")
.set("spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes", "100")
.set("spark.sql.adaptive.advisoryPartitionSizeInBytes", "100")
val spark = SparkSession.builder()
.config(sparkConf)
.getOrCreate()
spark.range(0, 1000, 1, 10)
.selectExpr("id as k1", "id as v1")
.createOrReplaceTempView("ta")
spark.range(0, 1000, 1, 10)
.selectExpr("id % 1 as k21", "id % 1 as k22", "id as v2")
.createOrReplaceTempView("tb")
spark.range(140)
.select(
col("id").cast("long").as("k3"),
concat(col("id").cast("string"), lit("a")).as("v3"))
.createOrReplaceTempView("tc")
spark.sql(
"""
|SELECT *
|FROM ta
|LEFT JOIN tb ON ta.k1 = tb.k21
|LEFT JOIN tc ON tb.k22 = tc.k3
|""".stripMargin)
.createOrReplaceTempView("v1")
spark.sql(
"""
|SELECT * FROM v1 WHERE v3 IS NOT NULL
|UNION
|SELECT * FROM v1
|""".stripMargin)
.collect()
```
This PR proposes a solution to address this problem. It introduces an asynchronous thread for the removal of the original file. Once the sorted file is generated for a given partition, this modification ensures that both non-range and range fetch requests will be able to and only fetch the sorted file once it is generated for a given partition.
this activity diagram of `openStream`

### Does this PR introduce _any_ user-facing change?
No, only bug fix
### How was this patch tested?
UT
Closes#1932 from cfmcgrady/fix-partition-sort-bug-v4.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
use `TimeUnit.NANOSECONDS.toMillis` instead of `/1000_000`
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1888 from cxzl25/CELEBORN-957.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
fix duplicated sending commitFiles for MapPartition and fix not sending BufferStreamEnd while opening MapPartition split.
### Why are the changes needed?
After open partition split for MapPartition, there are 2 errors.
- ERROR1 : Worker don't send streamend to client because concurrent thread sync problem . After idle timeout, client will close the channel and throws the Exception **" xx is lost, notify related stream xx"**
```java
2023-09-06T04:40:47.7549935Z 23/09/06 04:40:47,753 WARN [Keyed Aggregation -> Map -> Sink: Unnamed (5/8)#0] Task: Keyed Aggregation -> Map -> Sink: Unnamed (5/8)#0 (c1cade728ddb3a32e0bf72acb1d87588_c27dcf7b54ef6bfd6cff02ca8870b681_4_0) switched from RUNNING to FAILED with failure cause:
2023-09-06T04:40:47.7550644Z java.io.IOException: Client localhost/127.0.0.1:38485 is lost, notify related stream 256654410004
2023-09-06T04:40:47.7551219Z at org.apache.celeborn.plugin.flink.RemoteBufferStreamReader.errorReceived(RemoteBufferStreamReader.java:142)
2023-09-06T04:40:47.7551886Z at org.apache.celeborn.plugin.flink.RemoteBufferStreamReader.lambda$new$0(RemoteBufferStreamReader.java:77)
2023-09-06T04:40:47.7552576Z at org.apache.celeborn.plugin.flink.network.ReadClientHandler.processMessageInternal(ReadClientHandler.java:57)
2023-09-06T04:40:47.7553250Z at org.apache.celeborn.plugin.flink.network.ReadClientHandler.lambda$channelInactive$0(ReadClientHandler.java:119)
2023-09-06T04:40:47.7553806Z at java.util.concurrent.ConcurrentHashMap.forEach(ConcurrentHashMap.java:1597)
2023-09-06T04:40:47.7554564Z at org.apache.celeborn.plugin.flink.network.ReadClientHandler.channelInactive(ReadClientHandler.java:110)
2023-09-06T04:40:47.7555270Z at org.apache.celeborn.common.network.server.TransportRequestHandler.channelInactive(TransportRequestHandler.java:71)
2023-09-06T04:40:47.7556005Z at org.apache.celeborn.common.network.server.TransportChannelHandler.channelInactive(TransportChannelHandler.java:136)
2023-09-06T04:40:47.7556710Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:305)
2023-09-06T04:40:47.7557370Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7558172Z at io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
2023-09-06T04:40:47.7558803Z at io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
2023-09-06T04:40:47.7559368Z at io.netty.handler.timeout.IdleStateHandler.channelInactive(IdleStateHandler.java:277)
2023-09-06T04:40:47.7559954Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:303)
2023-09-06T04:40:47.7560589Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7561222Z at io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
2023-09-06T04:40:47.7561829Z at io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
2023-09-06T04:40:47.7562620Z at org.apache.celeborn.plugin.flink.network.TransportFrameDecoderWithBufferSupplier.channelInactive(TransportFrameDecoderWithBufferSupplier.java:206)
2023-09-06T04:40:47.7563506Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:305)
2023-09-06T04:40:47.7564207Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7564829Z at io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
2023-09-06T04:40:47.7565417Z at io.netty.channel.DefaultChannelPipeline$HeadContext.channelInactive(DefaultChannelPipeline.java:1405)
2023-09-06T04:40:47.7566014Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:301)
2023-09-06T04:40:47.7566654Z at io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
2023-09-06T04:40:47.7567317Z at io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:901)
2023-09-06T04:40:47.7567813Z at io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:813)
2023-09-06T04:40:47.7568297Z at io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:174)
2023-09-06T04:40:47.7568830Z at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:167)
2023-09-06T04:40:47.7569402Z at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470)
2023-09-06T04:40:47.7569894Z at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:569)
2023-09-06T04:40:47.7570356Z at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
2023-09-06T04:40:47.7570841Z at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
2023-09-06T04:40:47.7571319Z at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
2023-09-06T04:40:47.7571721Z at java.lang.Thread.run(Thread.java:750)
```
- ERROR2: Client will send duplicated commitFiles to worker. Becuase of inconsistency unHandledPartiitions , both batchCommit and finalCommit send commitFiles
``` java
2023-09-06T04:36:48.3146773Z 23/09/06 04:36:48,314 WARN [Worker-CommitFiles-1] Controller: Get Partition Location for 1693975002919-61094c8156f918062a5fae12d551bc90-0 0-1 but didn't exist.
```
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
ci
Closes#1881 from zhongqiangczq/fix-split-test.
Authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
### What changes were proposed in this pull request?
This PR fixes performance degradation when Spark's coalescePartitions takes effect caused
by RPC latency.
### Why are the changes needed?
I encountered a performance degradation when testing tpcds 10T q10:
||Time|
|---|---|
|ESS|14s|
|Celeborn| 24s|
After digging into it I found out that q10 triggers partition coalescence:

As I configured `spark.sql.adaptive.coalescePartitions.initialPartitionNum` to 1000, `CelebornShuffleReader`
will call `shuffleClient.readPartition` sequentially 1000 times, causing the delay.
This PR optimizes by calling `shuffleClient.readPartition` in parallel. After this PR q10 time becomes 14s.
### Does this PR introduce _any_ user-facing change?
No, but introduced a new client side configuration `celeborn.client.streamCreatorPool.threads`
which defaults to 32.
### How was this patch tested?
TPCDS 1T and passes GA.
Closes#1876 from waitinfuture/943.
Lead-authored-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
Co-authored-by: Keyong Zhou <waitinfuture@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
In MapPartiitoin, datas are split into regions.
1. Unlike ReducePartition whose partition split can occur on pushing data
to keep MapPartition data ordering, PartitionSplit only be done on the time of sending PushDataHandShake or RegionStart messages (As shown in the following image). That's to say that the partition split only appear at the beginnig of a region but not inner a region.
> Notice: if the client side think that it's failed to push HandShake or RegionStart messages. but the worker side can still receive normal HandShake/RegionStart message. After client revive succss, it don't push any messages to old partition, so the worker having the old partition will create a empty file. After committing files, the worker will return empty commitids. That's to say that empty file will be filterd after committing files and ReduceTask will not read any empty files.

2. PushData/RegioinFinish don't care the following cases:
- Diskfull
- ExceedPartitionSplitThreshold
- Worker ShuttingDown
so if one of the above three conditions appears, PushData and RegionFinish cant still do as normal. Workers should consider the ShuttingDown case and try best to wait all the regions finished before shutting down.
if PushData or RegionFinish failed like network timeout and so on, then MapTask will failed and start another attempte maptask.

3. how shuffle read supports partition split?
ReduceTask should get split paritions by order and open the stream by partition epoc orderly
### Why are the changes needed?
PartiitonSplit is not supported by MapPartition from now.
There still a risk that a partition file'size is too large to store the file on worker disk.
To avoid this risk, this pr introduces partition split in shuffle read and shuffle write.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
UT and manual TPCDS test
Closes#1550 from FMX/CELEBORN-627.
Lead-authored-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
Co-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Ethan Feng <ethanfeng@apache.org>
Signed-off-by: zhongqiang.czq <zhongqiang.czq@alibaba-inc.com>
### What changes were proposed in this pull request?
For spark clusters, support read local shuffle file if Celeborn is co-deployed with yarn node managers. This PR help to reduce the number of active connections.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and cluster. The performance is identical whether you enable local reader, but the active connection number may vary according to your connections per peer.
<img width="951" alt="截屏2023-08-16 20 20 14" src="https://github.com/apache/incubator-celeborn/assets/4150993/9106e731-28fc-4e78-9c05-ae6a269d249a">
The active connection number changed from 3745 to 2894. This PR will help to improve cluster stability.
Closes#1812 from FMX/CELEBORN-752.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
`CelebornShuffleFallbackPolicyRunner` could not only check quota, but also check whether cluster has available workers. If there is no available workers, fallback to external shuffle.
### Why are the changes needed?
`CelebornShuffleFallbackPolicyRunner` adds a check for available workers.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `SparkShuffleManagerSuite#testClusterNotAvailableWithAvailableWorkers`
Closes#1814 from SteNicholas/CELEBORN-830.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>