### What changes were proposed in this pull request?
Introduce JVM monitoring in Celeborn Worker using JVMQuake to enable early detection of memory management issues and facilitate fast failure.
### Why are the changes needed?
When facing out-of-control memory management in Celeborn Worker we typically use JVMkill as a remedy by killing the process and generating a heap dump for post-analysis. However, even with jvmkill protection, we may still encounter issues caused by JVM running out of memory, such as repeated execution of Full GC without performing any useful work during the pause time. Since the JVM does not exhaust 100% of resources, JVMkill will not be triggered. Therefore JVMQuake is introduced to provide more granular monitoring of GC behavior, enabling early detection of memory management issues and facilitating fast failure. Refers to the principle of [jvmquake](https://github.com/Netflix-Skunkworks/jvmquake) which is a JVMTI agent that attaches to your JVM and automatically signals and kills it when the program has become unstable.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`JVMQuakeSuite`
Closes#2061 from SteNicholas/CELEBORN-1092.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
SortBasedPusher `pushData` should inc memory spill metrics
### Why are the changes needed?
Make metrics more acurate
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Closes#2117 from AngersZhuuuu/CELEBORN-1143.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
1. Remove UNKNOWN_DISK from StorageInfo.
2. Enable load-aware slots allocation when there is HDFS.
### Why are the changes needed?
To support the application's config about available storage types.
### Does this PR introduce _any_ user-facing change?
no.
### How was this patch tested?
GA and Cluster.
Closes#2098 from FMX/B1081-1.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Add failureaccess shade.
### Why are the changes needed?
When test main branch, client got error like below:
```
Caused by: java.lang.NoClassDefFoundError: org/apache/celeborn/shaded/com/google/common/util/concurrent/internal/InternalFutureFailureAccess
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LoadingValueReference.<init>(LocalCache.java:3517)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LoadingValueReference.<init>(LocalCache.java:3521)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2170)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2081)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache.get(LocalCache.java:4019)
at org.apache.celeborn.shaded.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4933)
at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.replyGetReducerFileGroup(ReducePartitionCommitHandler.scala:283)
at org.apache.celeborn.client.commit.ReducePartitionCommitHandler.handleGetReducerFileGroup(ReducePartitionCommitHandler.scala:300)
at org.apache.celeborn.client.CommitManager.handleGetReducerFileGroup(CommitManager.scala:266)
at org.apache.celeborn.client.LifecycleManager.org$apache$celeborn$client$LifecycleManager$$handleGetReducerFileGroup(LifecycleManager.scala:628)
at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:314)
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: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?
Manual test.
Closes#2116 from onebox-li/shade-add-failureaccess.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
This PR introduce dynamic ConfigService at SystemLevel and TenantLevel, Dynamic configuration is a type of configuration that can be changed at runtime as needed. It can be used at system level/tenant level. When applying dynamic configuration, the priority order is as follows: tenant level overrides system level, which in turn overrides static configuration(CelebornConf). This means that if a configuration is defined at the tenant level, it will be used instead of the system level or static configuration(CelebornConf). If the tenant-level configuration is missing,
the system-level configuration will be used. If the system-level configuration is also missing, CelebornConf
will be used as the default value.
There are several other tasks related to this feature that will be implemented in the future.
- [ ] [Add isDynamic property for CelebornConf](https://issues.apache.org/jira/browse/CELEBORN-1051)
- [ ] [Support DB based Configserver](https://issues.apache.org/jira/browse/CELEBORN-1054)
- [ ] [Add restAPI for configuration management](https://issues.apache.org/jira/browse/CELEBORN-1056)
### Why are the changes needed?
The current configuration of the server (CelebornConf) is static. When the configuration is changed, the service needs to be restarted. This PR introduces a dynamic configuration solution. The server side can use dynamic configuration as needed. At the same time, it is considered that the tenant level will be supported in the future (such as supporting tenant level dynamic quota control) configuration, so this time we will also consider supporting dynamic tenant-level configuration, and this PR will provide a default implementation based on the file system.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT
Closes#2100 from RexXiong/CELEBORN-1052.
Authored-by: Shuang <lvshuang.tb@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.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>
### What changes were proposed in this pull request?
Added test suites for `RpcEnv`, `NettyRpcEnv`, and other related classes.
These are copied over from Apache Spark. Some of the UTs in Apache Spark required changes in the source code like [SPARK-39468](https://issues.apache.org/jira/browse/SPARK-39468) which I didn't copy over.
### Why are the changes needed?
The change adds unit tests.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Just adds UTs. The source code changes are minimal.
Closes#2107 from otterc/CELEBORN-1135.
Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.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?
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2073 from cxzl25/CELEBORN-856.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
`SlotsAllocator` supports policy for master to assign slots fallback to roundrobin with no available slots.
### Why are the changes needed?
When the selected workers have no available slots, the loadaware policy could throw `MasterNotLeaderException`. It's recommended to support policy for master to assign slots fallback to roundrobin with no available slots. Meanwhile, the situation that there is no available slots would occur when the partition size has increased a lot in a short period of time.
```
Caused by: org.apache.celeborn.common.haclient.MasterNotLeaderException: Master:xx.xx.xx.xx:9099 is not the leader. Suggested leader is Master:xx.xx.xx.xx:9099. Exception:bound must be positive.
at org.apache.celeborn.service.deploy.master.clustermeta.ha.HAHelper.sendFailure(HAHelper.java:58)
at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:236)
at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.applyOrElse(Master.scala:314)
... 7 more
Caused by: java.lang.IllegalArgumentException: bound must be positive
at java.util.Random.nextInt(Random.java:388)
at org.apache.celeborn.service.deploy.master.SlotsAllocator.roundRobin(SlotsAllocator.java:202)
at org.apache.celeborn.service.deploy.master.SlotsAllocator.offerSlotsLoadAware(SlotsAllocator.java:151)
at org.apache.celeborn.service.deploy.master.Master.$anonfun$handleRequestSlots$1(Master.scala:598)
at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:199)
at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:189)
at org.apache.celeborn.service.deploy.master.Master.handleRequestSlots(Master.scala:587)
at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.$anonfun$applyOrElse$12(Master.scala:314)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:233)
... 8 more
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`SlotsAllocatorSuiteJ#testAllocateSlotsWithNoAvailableSlots`
Closes#2108 from SteNicholas/CELEBORN-1136.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
`MasterNotLeaderException` corrects the suggested leader of exception message.
### Why are the changes needed?
When current peer isn't the leader of master and the leader is switching which cache isn't expired, the suggested leader of exception message in MasterNotLeaderException is confusing that the suggested leader is current peer. It's recommened to correct suggested leader of exception message for MasterNotLeaderException if current peer is equal to the suggested leader.
```
Caused by: org.apache.celeborn.common.haclient.MasterNotLeaderException: Master:xx.xx.xx.xx:9099 is not the leader. Suggested leader is Master:xx.xx.xx.xx:9099. Exception:bound must be positive.
at org.apache.celeborn.service.deploy.master.clustermeta.ha.HAHelper.sendFailure(HAHelper.java:58)
at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:236)
at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.applyOrElse(Master.scala:314)
... 7 more
Caused by: java.lang.IllegalArgumentException: bound must be positive
at java.util.Random.nextInt(Random.java:388)
at org.apache.celeborn.service.deploy.master.SlotsAllocator.roundRobin(SlotsAllocator.java:202)
at org.apache.celeborn.service.deploy.master.SlotsAllocator.offerSlotsLoadAware(SlotsAllocator.java:151)
at org.apache.celeborn.service.deploy.master.Master.$anonfun$handleRequestSlots$1(Master.scala:598)
at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:199)
at org.apache.celeborn.common.metrics.source.AbstractSource.sample(AbstractSource.scala:189)
at org.apache.celeborn.service.deploy.master.Master.handleRequestSlots(Master.scala:587)
at org.apache.celeborn.service.deploy.master.Master$$anonfun$receiveAndReply$1.$anonfun$applyOrElse$12(Master.scala:314)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at org.apache.celeborn.service.deploy.master.Master.executeWithLeaderChecker(Master.scala:233)
... 8 more
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2109 from SteNicholas/CELEBORN-1137.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
- bump guava from 14.0.1 to 32.1.3-jre
- refer to https://github.com/apache/spark/pull/26911, remove usages of Guava that no longer work in Guava 27/32, and replace with workalikes. After this PR, Celeborn no longer relies on a specific version of Guava, and is compatible with Guava 14/27/32. we have the ability to specify Guava to 27 when running MapReduce integration tests.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GA
Closes#2090 from cfmcgrady/guava-27.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Support fallback to non-columnar shuffle for schema that cannot be obtained from shuffle dependency.
### Why are the changes needed?
When columnar shuffle is enabled, it was found that the shuffle class operator of Spark RDD is not supported. It's recommended to support fallback to non-columnar shuffle for schema that cannot be obtained from shuffle dependency.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `CelebornColumnarShuffleReaderSuite#columnarShuffleReaderNewSerializerInstance`
- `ColumnarHashBasedShuffleWriterSuiteJ#createColumnarShuffleWriter`
Closes#2101 from gaochao0509/CELEBORN-1123.
Authored-by: gaochao0509 <1623735386@qq.com>
Signed-off-by: xiyu.zk <xiyu.zk@alibaba-inc.com>
### What changes were proposed in this pull request?
Celeborn Flink client validates whether `execution.batch-shuffle-mode` is `ALL_EXCHANGES_BLOCKING`.
### Why are the changes needed?
The config option `execution.batch-shuffle-mode` of Flink is `ALL_EXCHANGES_BLOCKING` by default. Celeborn Flink client should validate whether `execution.batch-shuffle-mode` is `ALL_EXCHANGES_BLOCKING`. If `execution.batch-shuffle-mode` is set as `ALL_EXCHANGES_PIPELINED`, there is `NullPointerException` for `ReducePartitionCommitHandler#handleGetReducerFileGroup`, which exception is as follows:
```
023-11-16 14:40:55,984 ERROR org.apache.celeborn.common.rpc.netty.Inbox - Ignoring error
java.lang.NullPointerException: Cannot invoke "java.util.Set.add(Object)" because the return value of "java.util.concurrent.ConcurrentHashMap.get(Object)" is 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:559)
at org.apache.celeborn.client.LifecycleManager$$anonfun$receiveAndReply$1.applyOrElse(LifecycleManager.scala:297)
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.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`RemoteShuffleServiceFactorySuitJ#testInvalidShuffleServiceConfig`.
Closes#2106 from SteNicholas/CELEBORN-1134.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Change WorkerInfo#hashCode() from map+foldLeft to while and cache.
Test the each way to calculate, code and result show as below:
```
val state = Seq(host, rpcPort, pushPort, fetchPort, replicatePort)
// origin
val originHash = state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b)
// for
var forHash = 0
for (i <- state) {
forHash = 31 * forHash + i.hashCode()
}
// while
var whileHash = 0
var i = 0
while (i < state.size) {
whileHash = 31 * whileHash + state(i).hashCode()
i = i + 1
}
```
Result:
```
java version "1.8.0_261"
origin hash result = -831724440, costs 1103914 ns
for hash result = -831724440, costs 444588 ns (2.5x)
while hash result = -831724440, costs 46510 ns (23x)
```
### Why are the changes needed?
The current WorkerInfo's hashCode() is a little time-consuming. Since it is widely used in lots of hash maps, it needs to be improved.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Added UT.
Closes#2086 from onebox-li/improve-worker-hash.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: Fu Chen <cfmcgrady@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?
For the `helm` chart. I specified the resources field for the `initContainers` for `worker` and `master` statefulsets.
I used the same values which are specified for the "main" container
### Why are the changes needed?
For users that have a `ResourceQuota` (such as myself), worker and master pods do not start since the initContainers for the statefulsets do not specify the resources (cpu/ memory requests and limits).
### Does this PR introduce _any_ user-facing change?
### [Issue](https://github.com/apache/incubator-celeborn/issues/2094)
### How was this patch tested?
I installed the chart on my GKE cluster.
Closes#2093 from marwansalem/set-k8s-resources-for-init-containers.
Authored-by: Marwan Salem <marwan.saad@incorta.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
The changes are to ensure that the data is at least written into the flush buffer before sending a message back to the client. Earlier, the message would be sent before this happens.
### Why are the changes needed?
Changes are needed because currently the primary will send a response back to client before it is even written into the flush buffer to persist locally. We do this persist async. Additionally, this will prevent data corruption issues when data may not be present properly in primary but only on replica, but client fetches only from primary.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Will let CI run, and also tested on our internal cluster
Closes#2064 from akpatnam25/CELEBORN-1106.
Lead-authored-by: Aravind Patnam <apatnam@linkedin.com>
Co-authored-by: Aravind Patnam <akpatnam25@gmail.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?
Followup support `celeborn.worker.storage.disk.reserve.ratio` with `minimumUsableSize` cache in a variable instead of calculate for every pushdata.
### Why are the changes needed?
Cache `minimumUsableSize` in a variable instead of calculate for every pushdata because `DiskUtils.getMinimumUsableSize` is costly.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`SlotsAllocatorSuiteJ`
Closes#2083 from SteNicholas/CELEBORN-1110.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
As title.
### Why are the changes needed?
To avoid users being confused after upgrading.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#2087 from onebox-li/slots_allocated_metric_panel.
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?
Add the following patch files in directory `incubator-celeborn/tree/spark3-patch/assets/spark-patch` :
1. Celeborn_Dynamic_Allocation_spark3_0.patch
2. Celeborn_Dynamic_Allocation_spark3_1.patch
3. Celeborn_Dynamic_Allocation_spark3_2.patch
4. Celeborn_Dynamic_Allocation_spark3_3.patch
Delete a patch at the same time:
1. Celeborn_Dynamic_Allocation_spark3.patch
Modified `Support Spark Dynamic Allocation` in incubator-celeborn/README.md :

### Why are the changes needed?
Convenient for customers to apply patches in Spark 3.X for `Support Spark Dynamic Allocation`
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
yes. All patch files can be applied to the corresponding version of spark source code through `git apply` without any code conflicts.
Closes#2085 from lukeyan2023/spark3-patch.
Authored-by: Luke Yan <108530647+lukeyan2023@users.noreply.github.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.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?
1. Make Celeborn read configs from HADOOP_COND_DIR.
2. Remove unnecessary Kerberos configs.
### Why are the changes needed?
To support HDFS with Kerberos.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and cluster.
Closes#2082 from FMX/B1116.
Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Fu Chen <cfmcgrady@gmail.com>
Co-authored-by: Cheng Pan <pan3793@gmail.com>
Co-authored-by: Ethan Feng <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>
### What changes were proposed in this pull request?
add --release parameter to create a Celeborn distribution like those distributed by the Celeborn Downloads page
### Why are the changes needed?
Without --release parameter, the created Celeborn distribution is different from the Celeborn Downloads page and lacks client-related packages.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
PASS GA
Closes#2080 from jiaoqingbo/minor-sbt.
Authored-by: jiaoqingbo <1178404354@qq.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.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?
Currently, `WorkerInfo` is used in many places, and allocationBuckets is only used when its own workers want to collect metrics `SLOTS_ALLOCATED`. If there are lots of workers in the RSS cluster, there may be a certain amount of memory waste, each `WorkerInfo` maintain a Array\[Int](61), so remove it from `WorkerInfo`.
And refactor the metrics `SLOTS_ALLOCATED` from gauge to counter. Originally, this metrics is approximately one hour's total only if there are continuous tasks. Now refactoring it into a counter can reduce the cost of maintaining time windows, including storage and timely expiration data, etc. It can also be more flexibly transformed according to user needs on the prometheus side.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
Yes. metrics_SlotsAllocated_Count metrics change from gauge for 1 hour to a increasing counter.
### How was this patch tested?
Cluster test.
Closes#2078 from onebox-li/improve-SlotsAllocated.
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?
Because now we support Flink 1.18.
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2079 from cxzl25/CELEBORN-1108-FOLLOWOUP.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Shaoyun Chen <csy@apache.org>
### What changes were proposed in this pull request?
Support `celeborn.worker.storage.disk.reserve.ratio` to configure worker reserved ratio for each disk.
### Why are the changes needed?
`CelebornConf` supports to configure celeborn worker reserved space for each disk, which space is absolute. `CelebornConf` could support `celeborn.worker.storage.disk.reserve.ratio` to configure worker reserved ratio for each disk. The minimum usable size for each disk should be the max space between the reserved space and the space calculate via reserved ratio.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`SlotsAllocatorSuiteJ`
Closes#2071 from SteNicholas/CELEBORN-1110.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@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?
Support exclude worker manually given worker id. This worker is added into excluded workers manually.
### Why are the changes needed?
Celeborn supports to shuffle client-side fetch and push exclude workers on failure at present. It's necessary to exclude worker manually for maintaining the Celeborn cluster.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `HttpUtilsSuite`
- `DefaultMetaSystemSuiteJ#testHandleWorkerExclude`
- `RatisMasterStatusSystemSuiteJ#testHandleWorkerExclude`
- `MasterStateMachineSuiteJ#testObjSerde`
Closes#1997 from SteNicholas/CELEBORN-448.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Update log level for `TransferBufferPool#reserveBuffers` from warn to debug.
### Why are the changes needed?
The log level of `TransferBufferPool#reserveBuffers` should not be warn, which is normal credit-based shuffle read log.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#2076 from SteNicholas/reserve-buffers-log.
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?
```bash
flink-1.18.0
./bin/start-cluster.sh
./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
```
```java
Caused by: java.lang.NoSuchMethodError: org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.<init>(Ljava/lang/String;ILorg/apache/flink/runtime/jobgraph/IntermediateDataSetID;Lorg/apache/flink/runtime/io/network/partition/ResultPartitionType;Lorg/apache/flink/runtime/executiongraph/IndexRange;ILorg/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider;Lorg/apache/flink/util/function/SupplierWithException;Lorg/apache/flink/runtime/io/network/buffer/BufferDecompressor;Lorg/apache/flink/core/memory/MemorySegmentProvider;ILorg/apache/flink/runtime/throughput/ThroughputCalculator;Lorg/apache/flink/runtime/throughput/BufferDebloater;)V
at org.apache.celeborn.plugin.flink.RemoteShuffleInputGate$FakedRemoteInputChannel.<init>(RemoteShuffleInputGate.java:225)
at org.apache.celeborn.plugin.flink.RemoteShuffleInputGate.getChannel(RemoteShuffleInputGate.java:179)
at org.apache.flink.runtime.io.network.partition.consumer.InputGate.setChannelStateWriter(InputGate.java:90)
at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.setChannelStateWriter(InputGateWithMetrics.java:120)
at org.apache.flink.streaming.runtime.tasks.StreamTask.injectChannelStateWriterIntoChannels(StreamTask.java:524)
at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:496)
```
Flink 1.18.0 release
https://flink.apache.org/2023/10/24/announcing-the-release-of-apache-flink-1.18/
Interface `org.apache.flink.runtime.io.network.buffer.Buffer` adds `setRecycler` method.
[[FLINK-32549](https://issues.apache.org/jira/browse/FLINK-32549)][network] Tiered storage memory manager supports ownership transfer for buffers
`org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate` constructor adds parameters.
[[FLINK-31638](https://issues.apache.org/jira/browse/FLINK-31638)][network] Introduce the TieredStorageConsumerClient to SingleInputGate
[[FLINK-31642](https://issues.apache.org/jira/browse/FLINK-31642)][network] Introduce the MemoryTierConsumerAgent to TieredStorageConsumerClient
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
```bash
flink-1.18.0 ./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
Executing example with default input data.
Use --input to specify file input.
Printing result to stdout. Use --output to specify output path.
Job has been submitted with JobID d7fc5f0ca018a54e9453c4d35f7c598a
Program execution finished
Job with JobID d7fc5f0ca018a54e9453c4d35f7c598a has finished.
Job Runtime: 1635 ms
```
<img width="1297" alt="image" src="https://github.com/apache/incubator-celeborn/assets/3898450/6a5266bf-2386-4386-b98b-a60d2570fa99">
Closes#2063 from cxzl25/CELEBORN-1105.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>
### What changes were proposed in this pull request?
Flink/Spark jars packaged with SBT use the correct LICENSE and NOTICE.
### Why are the changes needed?
https://github.com/apache/incubator-celeborn/pull/1930#discussion_r1340410526
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#1967 from cxzl25/CELEBORN-1031.
Lead-authored-by: sychen <sychen@ctrip.com>
Co-authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.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?
Adding Kerberos support for HDFS storage type.
The following five parameters need to be configured:
| key | value |
| :--: | :--: |
| celeborn.storage.hdfs.kerberos.enabled | true |
| celeborn.storage.hdfs.kerberos.principal | userREALM |
| celeborn.storage.hdfs.kerberos.keytab | /path/test.keytab |
| celeborn.hadoop.hadoop.security.authorization | kerberos |
| celeborn.hadoop.dfs.namenode.kerberos.principal | hdfs/_HOSTREALM |
### Why are the changes needed?
Connecting to HDFS with Kerberos enabled requires support for keytab login.
### Does this PR introduce _any_ user-facing change?
Add 3 configurations.
celeborn.storage.hdfs.kerberos.enabled
celeborn.storage.hdfs.kerberos.principal
celeborn.storage.hdfs.kerberos.keytab
### How was this patch tested?
Test in Kerberos enabled HDFS cluster.
Closes#2072 from liujiayi771/hdfs-kerberos.
Authored-by: joey.ljy <joey.ljy@alibaba-inc.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?
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2068 from cxzl25/CELEBORN-1108.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
This change makes the maximum default number of Netty threads configurable. Previously, this value was hardcoded to 64, which could be small for certain environments. While it's possible to configure the number of Netty server and client threads individually for each module, providing an option to increase the default value offers greater convenience.
### Why are the changes needed?
The change offers convenience.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added a UT
Closes#2065 from otterc/CELEBORN-1107.
Authored-by: Chandni Singh <singh.chandni@gmail.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.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?
The `cleaner` of `Worker` executes the `StorageManager#cleanupExpiredShuffleKey` to clean expired shuffle keys with daemon cached thread pool. The optimization speeds up cleaning including expired shuffle keys of ChunkManager to avoid memory leak.
### Why are the changes needed?
`ChunkManager#streams` could lead memory leak when the speed of cleanup is slower than expiration for expired shuffle of worker. The behavior that `ChunkStreamManager` cleanup expired shuffle key should be optimized to avoid memory leak, which causes that the VM thread of worker is 100%.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`WorkerSuite#clean up`.
Closes#2053 from SteNicholas/CELEBORN-1094.
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#2062 from cxzl25/CELEBORN-1104.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Add the metric `ResourceConsumption` to monitor each user's quota usage of Celeborn Worker.
### Why are the changes needed?
The metric `ResourceConsumption` supports to monitor each user's quota usage of Celeborn Master at present. The usage of Celeborn Worker also needs to monitor.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Internal tests.
Closes#2059 from SteNicholas/CELEBORN-247.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@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?
Seperate `overHighWatermark` check to a dedicated thread, let this value can shared better and lighten `CongestionController#isUserCongested` logic.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test and UT.
Closes#2041 from onebox-li/congest-check.
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 a bad disk occurs, cleaning up expired shuffle keys can cause NullPointerException appearing in the thread pool obtained from `diskOperators` in `StorageManager`.
Therefore, only cleaning up expired shuffle keys from good disks will not cause the above problems.**
https://issues.apache.org/jira/browse/CELEBORN-1103
### Why are the changes needed?
bugfix
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#2060 from suizhe007/CELEBORN-1103.
Lead-authored-by: qinrui <qr7972@gmail.com>
Co-authored-by: qinrui <51885730+suizhe007@users.noreply.github.com>
Signed-off-by: zky.zhoukeyong <zky.zhoukeyong@alibaba-inc.com>