### What changes were proposed in this pull request?
1. Worker reports resourceConsumption to master
2. QuotaManager calculates the resourceConsumption of each app and marks the apps that exceed the quota.
2.1 When the tenant's resourceConsumption exceeds the tenant's quota, select the app with a larger consumption to mark interrupted.
2.2 When the resourceConsumption of the cluster exceeds the cluster quota, select the app with larger consumption to mark interrupted.
3. Master returns to Driver through heartbeat, whether app is marked interrupted
### Why are the changes needed?
The current storage quota logic can only limit new shuffles, and cannot limit the writing of existing shuffles. In our production environment, there is such an scenario: the cluster is small, but the user's app single shuffle is large which occupied disk resources, we want to interrupt those shuffle.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
UTs.
Closes#2819 from leixm/CELEBORN-1577-2.
Authored-by: Xianming Lei <31424839+leixm@users.noreply.github.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix incorrect logic when calculate disk available slots
### Why are the changes needed?
Now we use `usableSize / estimatedPartitionSize = maxSlots`
Then `availableSlots = maxSlots - allocatedSlots`
But `availableSlots` should be `usableSize / estimizatedPartitionSize`
### Does this PR introduce _any_ user-facing change?
Yea
### How was this patch tested?
MT
Closes#3162 from AngersZhuuuu/CELEBORN-1923.
Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Whenever a `WorkerPartitionReader` is recreated (due celeborn worker restarts / any other chunk fetch failure), the entire shuffle partition file is re-read from beginning, discarding already read chunks in `CelebornInputStream` based on the batchIdSet metadata maintained.
This can be improved (only for cases where shuffle data is unreplicated) by skipping already read chunk id since they'd be discarded anyway. This improves overall shuffle read performance (reducer's total time, network usage etc).
### Why are the changes needed?
Allow skipping already read shuffle chunks
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UTs added
Closes#3132 from saurabhd336/skipReadChunks.
Authored-by: Saurabh Dubey <saurabhd336@uber.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Support pre-run static code blocks of `TransportMessages` to improve performance of protobuf serialization.
### Why are the changes needed?
The protobuf message protocol defines many map type fields, which makes it time-consuming to build these message instances. This is because `TransportMessages` contains static code blocks to initialize a large number of `Descriptor`s and `FieldAccessorTable`s, where the instantiation of `FieldAccessorTable` includes reflection. The test result proves that the static code blocks execute in about 70 milliseconds.
Therefore, it's better to pre-run static code blocks of `TransportMessages` to improve performance of protobuf serialization. Meanwhile, it's recommended to use repeated instead of map type field for rpc messages.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3149 from SteNicholas/CELEBORN-1909.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
LifecycleManager respond to RegisterShuffle with max epoch PartitionLocation.
### Why are the changes needed?
Newly spun up executors in a Spark job will still get the partitionLocations with the minEpoch of the celeborn lost worker.
These executors will fail to connect to the lost worker and then call into revive to get the latest PartitionLocation for a given partitionId in `ChangePartitionManager.getLatestPartition()`.
Return with max epoch can reduce such revive requests.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT.
Closes#3135 from zaynt4606/clb1822.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Remove the redundant synchronized keyword from the `isTerminated` method in `sameThreadExecutorService()` implementation within the `ThreadUtils` class. The method was using both a synchronized block and an explicit `ReentrantLock`, which is unnecessary and potentially problematic.
Backport https://github.com/apache/spark/pull/50210.
### Why are the changes needed?
The changes are needed for several reasons:
1. **Eliminates redundant synchronization**: The current implementation uses both synchronized keyword and explicit ReentrantLock, which is redundant and creates unnecessary overhead.
2. **Reduces potential deadlock risks**: Using two different locking mechanisms (built-in synchronized and explicit `ReentrantLock`) in the same method could lead to complex lock ordering issues and increase deadlock risks. Although the risk of deadlock in the current implementation is low, if someone modifies the code in the future and adds a method that acquires these two locks in a different order, it would introduce a deadlock risk.
3. **Improves performance**: Removing the unnecessary synchronization layer reduces lock contention and context switching overhead.
4. **Code consistency**: Other methods in the same class only use `ReentrantLock` for synchronization, so removing synchronized makes the code style more consistent.
5. **More precise control**: `ReentrantLock` already provides all the synchronization needed with more features than the implicit synchronized keyword.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3150 from SteNicholas/CELEBORN-1910.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
close [issues-3145](https://github.com/apache/celeborn/issues/3145)
### Why are the changes needed?
1. Support s3 schema.
2. Fixed the problem that diskFileInfo judged the S3 type incorrectly.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Deployment integration testing has been completed in the local environment.
Closes#3146 from shouwangyw/bugfix/resolve_bugs_3145.
Authored-by: veli.yang <897900564@qq.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
0.6.0 -> 0.5.4
- `celeborn.rpc.retryWait`
- `celeborn.client.rpc.retryWait`
`empty` -> 0.5.4
- `celeborn.<module>.io.conflictAvoidChooser.enable`
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#3142 from s0nskar/config_rpc_retry.
Authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
196ad607cd/common/src/main/scala/org/apache/celeborn/common/rpc/netty/NettyRpcEnv.scala (L256) we hit the issue that the ignored message is too long , and when calling toString, it applies for a too large array which is beyond the JVM's limit
I don't think this log convey too much info, , so we could avoid calling toString to improve robustness of the applications
### Why are the changes needed?
more details in JIRA
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
prod
Closes#3139 from CodingCat/log.
Authored-by: CodingCat <zhunansjtu@gmail.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Minor change for the log, to make the fetch failure message and rpc metrics dump much clear.
### Why are the changes needed?
As title.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
GA.
Closes#3136 from turboFei/log_shuffle.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
In the switchServingState after resumeByPinnedMemory, keep the resume channel to prevent the channel from frequently resuming and pausing before memoryUsage decreases to pausePushDataThreshold.
### Why are the changes needed?
Frequent channel resume and pause will result in slow data reception and failure to quickly reduce memoryUsage to below pausePushDataThreshold.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
ut
Closes#3099 from TheodoreLx/keep-resume.
Lead-authored-by: TheodoreLx <1548069580@qq.com >
Co-authored-by: 慧枫 <zhengqi.zzq@antgroup.com>
Co-authored-by: Zhengqi Zhang <1548069580@qq.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Followup for https://github.com/apache/celeborn/pull/2373,
fix the `IndexOutOfBoundsException` when using old celeborn client.
### Why are the changes needed?
I meet below exception when using old celeborn client, seem incompatibility issue.
The error log in worker end:
```
25/02/27 14:18:24,182 ERROR [rpc_service-server-4-6] NettyRpcHandler: Error while invoking NettyRpcHandler#receive() on RPC id 4330
java.lang.IndexOutOfBoundsException: Index:0, Size:0
at com.google.protobuf.LongArrayList.ensureIndexInRange(LongArrayList.java:265)
at com.google.protobuf.LongArrayList.getLong(LongArrayList.java:113)
at org.apache.celeborn.common.protocol.PbPackedPartitionLocations.getFileSizes(PbPackedPartitionLocations.java:465)
at org.apache.celeborn.common.util.PbSerDeUtils$.fromPackedPartitionLocations(PbSerDeUtils.scala:653)
at org.apache.celeborn.common.util.PbSerDeUtils$.fromPbPackedPartitionLocationsPair(PbSerDeUtils.scala:591)
at org.apache.celeborn.common.protocol.message.ControlMessages$.fromTransportMessage(ControlMessages.scala:1312)
at org.apache.celeborn.common.util.Utils$.fromTransportMessage(Utils.scala:1056)
at org.apache.celeborn.common.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:110)
at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.$anonfun$deserialize$2(NettyRpcEnv.scala:313)
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62)
at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:363)
at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.$anonfun$deserialize$1(NettyRpcEnv.scala:312)
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62)
at org.apache.celeborn.common.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:312)
at org.apache.celeborn.common.rpc.netty.RequestMessage$.apply(NettyRpcEnv.scala:555)
at org.apache.celeborn.common.rpc.netty.NettyRpcHandler.internalReceive(NettyRpcEnv.scala:619)
at org.apache.celeborn.common.rpc.netty.NettyRpcHandler.receive(NettyRpcEnv.scala:605)
at org.apache.celeborn.common.network.server.TransportRequestHandler.processRpcRequest(TransportRequestHandler.java:101)
at org.apache.celeborn.common.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:85)
at org.apache.celeborn.common.network.server.TransportChannelHandler.channelRead(TransportChannelHandler.java:156)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:289)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at org.apache.celeborn.common.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:74)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1357)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:868)
at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:799)
at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:501)
at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:399)
at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:998)
at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:833)
25/02/27 14:18:24,234 WARN [celeborn-dispatcher-81] Controller: Shuffle application_1739172886147_152994_1-0 not registered!
25/02/27 14:18:24,284 WARN [celeborn-dispatcher-82] Controller: Shuffle application_1739172886147_152994_1-0 not registered!
25/02/27 14:18:24,335 WARN [celeborn-dispatcher-83] Controller: Shuffle application_1739172886147_152994_1-0 not registered!
```
The client end log:
```
org.apache.celeborn.common.exception.CelebornIOException: Register shuffle failed for shuffle 0, reason: RESERVE_SLOTS_FAILED
at org.apache.celeborn.client.ShuffleClientImpl.registerShuffleInternal(ShuffleClientImpl.java:710)
at org.apache.celeborn.client.ShuffleClientImpl.registerShuffle(ShuffleClientImpl.java:519)
at org.apache.celeborn.client.ShuffleClientImpl.lambda$getPartitionLocation$4(ShuffleClientImpl.java:579)
at java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1708)
at org.apache.celeborn.client.ShuffleClientImpl.getPartitionLocation(ShuffleClientImpl.java:575)
at org.apache.celeborn.client.write.DataPushQueue.takePushTasks(DataPushQueue.java:92)
at org.apache.celeborn.client.write.DataPusher$1.run(DataPusher.java:124)
at java.base/java.lang.Thread.run(Thread.java:833)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UT and IT.
Closes#3122 from turboFei/index_outof.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Support configuring the SSL handshake timeout for SSLHandler, for `rpc_app` and `rpc_service` transport modules.
### Why are the changes needed?
To make the SSLHandler handshake timeout configurable. We are working on ramping shuffle traffic to Celeborn internally, and have observed spark task failures which related to the connection timeout. This will make SSLHandler handshake timeout in line with our ecosystem’s production config, and should minimize those failures and improve robustness.
### Does this PR introduce _any_ user-facing change?
Introduces a new server side configuration.
### How was this patch tested?
Added a new UT, validated with existing UTs.
Closes#3120 from rmcyang/rmcyang/CELEBORN-1882.
Authored-by: Minchu Yang <minyang@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
Add remote address to RPC exception logs for `TransportRequestHandler`.
### Why are the changes needed?
`TransportRequestHandler` does not log the remote address on RPC exception at present. It's better to simplify troubleshooting via logging the remote address.
Backport https://github.com/apache/spark/pull/49718.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#3113 from SteNicholas/CELEBORN-1876.
Authored-by: Nicholas Jiang <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
`rpcEndpointRef` should be set to `null` When master leader is not present.
### Why are the changes needed?
Attempt master address index can only be updated when `rpcEndpointRef` is `null`.
The index wont update if `rpcEndpointRef` has been set but some error ocurs to the master leader.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Closes#3101 from zaynt4606/clb1865.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Add logic to support avoid sorting shuffle files for Reduce mode when optimize skew partitions
### Why are the changes needed?
Current logic need sorting shuffle files when read Reduce mode skew partition shuffle files, we found some shuffle sorting timeout and performance issue
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Cluster test and uts
Closes#2373 from wangshengjie123/optimize-skew-partition.
Lead-authored-by: wangshengjie <wangshengjie3@xiaomi.com>
Co-authored-by: wangshengjie3 <wangshengjie3@xiaomi.com>
Co-authored-by: Fu Chen <cfmcgrady@gmail.com>
Co-authored-by: Shuang <lvshuang.tb@gmail.com>
Co-authored-by: wangshengjie3 <soldier.sj.wang@gmail.com>
Co-authored-by: Fei Wang <fwang12@ebay.com>
Co-authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Support `celeborn.worker.storage.baseDir.diskType` option to specify disk type of base directory for worker.
### Why are the changes needed?
The disk type of base directory for worker is `HDD` at default. We could support `celeborn.worker.storage.baseDir.diskType` option to specify disk type of base directory for worker.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3096 from SteNicholas/CELEBORN-1861.
Authored-by: Nicholas Jiang <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Retry seding RPC to LifecycleManager when TimeoutException.
### Why are the changes needed?
RPC messages are processed by `Dispatcher.threadpool` which its numThreads depends on `numUsableCores`.
In some cases (k8s) the numThreads of LifecycleManager are not enough while the RPCs are a lot so there are TimeoutExceptions.
Add retry when there are TimeoutExceptions.
### Does this PR introduce _any_ user-facing change?
No.
Another way is to adjust the configuration `celeborn.lifecycleManager.rpc.dispatcher.threads` to add the numThreads.
This way is more affective.
### How was this patch tested?
Cluster testing.
Closes#3008 from zaynt4606/clb1757.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Remove unused `celeborn.<module>.io.enableVerboseMetrics` option.
### Why are the changes needed?
`celeborn.<module>.io.enableVerboseMetrics` option is unused, which could be replaced with `celeborn.network.memory.allocator.verbose.metric`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`CelebornConfSuite`
Closes#3094 from SteNicholas/CELEBORN-1860.
Authored-by: Nicholas Jiang <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Support custom implementation of EventExecutorChooser to avoid deadlock when calling await in EventLoop thread
### Why are the changes needed?
In Flink Celeborn Client, you can create a new connection in the EventLoop thread. To wait for the connection to complete, cf.await is called, which can cause a deadlock because the thread bound to the newly connected channel may be the same as the current EventLoop thread. The current thread is suspended by wait and needs to wait for the current thread to notify. This change is to avoid binding the same thread.
### Does this PR introduce _any_ user-facing change?
celeborn.<module>.io.conflictAvoidChooser.enable is introduced.
### How was this patch tested?
manual test
Closes#3071 from littlexyw/fix_dead_lock.
Authored-by: xinyuwang1 <xinyuwang1@xiaohongshu.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
0.6.0 -> 0.5.2
`celeborn.master.endpoints.resolver`
0.6.0 -> 0.5.1
`celeborn.client.chunk.prefetch.enabled`
`celeborn.client.inputStream.creation.window`
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#3082 from cxzl25/config_version.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Shaoyun Chen <csy@apache.org>
### What changes were proposed in this pull request?
Congestion and MemoryManager should use pinnedDirectMemory instead of usedDirectMemory
### Why are the changes needed?
In our production environment, after worker pausing, the usedDirectMemory keep high and does not decrease. The worker node is permanently blacklisted and cannot be used.
This problem has been bothering us for a long time. When the thred cache is turned off, in fact, **after ctx.channel().config().setAutoRead(false), the netty framework will still hold some ByteBufs**. This part of ByteBuf result in a lot of PoolChunks cannot be released.
In netty, if a chunk is 16M and 8k of this chunk has been allocated, then the pinnedMemory is 8k and the activeMemory is 16M. The remaining (16M-8k) memory can be allocated, but not yet allocated, netty allocates and releases memory in chunk units, so the 8k that has been allocated will result in 16M that cannot be returned to the operating system.
Here are some scenes from our production/test environment:
We config 10gb off-heap memory for worker, other configs as below:
```
celeborn.network.memory.allocator.allowCache false
celeborn.worker.monitor.memory.check.interval 100ms
celeborn.worker.monitor.memory.report.interval 10s
celeborn.worker.directMemoryRatioToPauseReceive 0.75
celeborn.worker.directMemoryRatioToPauseReplicate 0.85
celeborn.worker.directMemoryRatioToResume 0.5
```
When receiving high traffic, the worker's usedDirectMemory increases. After triggering trim and pause, usedDirectMemory still does not reach the resume threshold, and worker was excluded.

So we checked the heap snapshot of the abnormal worker, we can see that there are a large number of DirectByteBuffers in the heap memory. These DirectByteBuffers are all 4mb in size, which is exactly the size of chunksize. According to the path to gc root, DirectByteBuffer is held by PoolChunk, and these 4m only have 160k pinnedBytes.


There are many ByteBufs that are not released

The stack shows that these ByteBufs are allocated by netty

We tried to reproduce this situation in the test environment. When the same problem occurred, we added a restful api of the worker to force the worker to resume. After the resume, the worker returned to normal, and PushDataHandler handled many delayed requests.


So I think that when pinnedMemory is not high enough, we should not trigger pause and congestion, because at this time a large part of the memory can still be allocated.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#3018 from leixm/CELEBORN-1792.
Authored-by: Xianming Lei <31424839+leixm@users.noreply.github.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
1. Replace waitThreadPoll's thread pool with ScheduledExecutorService.
2. commitFile should reply when shuffleCommitTimeout.
### Why are the changes needed?
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Cluster test & UT.
Closes#3059 from zaynt4606/clb1829.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
`MapPartitionData` creates fixed thread pool with registration of `ThreadPoolSource`.
### Why are the changes needed?
`MapPartitionData` creates fixed thread pool without registering `ThreadPoolSource` at present, which causes that map partition reader thread of worker is lack of thread pool metrics. Therefore, `MapPartitionData` should create fixed thread pool with registration of `ThreadPoolSource`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3064 from SteNicholas/CELEBORN-1832.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Change `celeborn.<module>.io.mode` optional to explain default behavior in description.
### Why are the changes needed?
The default value of `celeborn.<module>.io.mode` in document could be changed by whether epoll mode is available for different os. Therefore, `celeborn.<module>.io.mode` should be changed to optional and explained the default behavior in description of option.
Follow up https://github.com/apache/celeborn/pull/3039#discussion_r1899340272.
### Does this PR introduce _any_ user-facing change?
`celeborn.<module>.io.mode` is optional and explains default behavior in description.
### How was this patch tested?
CI.
Closes#3044 from SteNicholas/CELEBORN-1774.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
This PR introduces a configuration `celeborn.network.memory.allocator.pooled` to allow users to disable `PooledByteBufAllocator` globally and always use `UnpooledByteBufAllocator`.
### Why are the changes needed?
In some extreme cases, the Netty's `PooledByteBufAllocator` might have tons of 4MiB chunks but only a few sizes of the capacity are used by the real data(see https://github.com/apache/celeborn/pull/3018), for scenarios that stability is important than performance, it's desirable to allow users to disable the `PooledByteBufAllocator` globally.
### Does this PR introduce _any_ user-facing change?
Add a new feature, disabled by default.
### How was this patch tested?
Pass UT to ensure correctness. Performance and memory impact need to be verified in the production scale cluster.
Closes#3043 from pan3793/CELEBORN-1815.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
To avoid possible worker load skew for the stages with tiny reducer numbers.
### Why are the changes needed?
If a stage has tiny reducers and skewed partitions, The default value will lead to serious worker load imbalance cause some workers unable to handle shuffle data.
### Does this PR introduce _any_ user-facing change?
Yes
### How was this patch tested?
GA and cluster test.
Closes#3039 from FMX/1811.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
To support Spark 4.0.0 preview.
### Why are the changes needed?
1. Changed Scala to 2.13.
2. Introduce columnar shuffle module for spark 4.0.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Cluster test.
Closes#2813 from FMX/b1413.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
1. rename the RPC metrics name from `${name}_${metric}` to `Rpc${metric}{name=$name}` so that it is easy to add into grafana dashboard
2. Use MASTER/WORKER/CLIENT Role for rpc env.
3. add the rpc metrics into grafana dashboard.
### Why are the changes needed?
For monitoring
### Does this PR introduce _any_ user-facing change?
No, it has not been released
### How was this patch tested?
UT for metrics source `instance`.
<img width="1456" alt="image" src="https://github.com/user-attachments/assets/90284390-54ad-49ef-a868-fa537d2301b8">
<img width="1880" alt="image" src="https://github.com/user-attachments/assets/e8101e47-d649-4c66-9978-1efb4faa047f">
Closes#2990 from turboFei/rpc_metrics.
Lead-authored-by: Wang, Fei <fwang12@ebay.com>
Co-authored-by: Fei Wang <cn.feiwang@gmail.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
1. Cancel all commit file jobs when handleCommitFiles timeout.
2. Fix timeout commit jobs wont be set `CommitInfo.COMMIT_FINISHED`
### Why are the changes needed?
1. Pending task in commitThreadPool wont be canceled.
3. Timeout commit jobs should set `commitInfo.status`.

### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
UT test.
The `commitInfo.status` should be `COMMIT_FINISHED` when commitFile jobs timeout.
Cluster test.

Closes#3004 from zaynt4606/clb1783.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Update default value of `celeborn.<module>.io.mode` to whether epoll mode is available. Meanwhile, the io mode of transport is `NIO` for unavailable epoll mode.
### Why are the changes needed?
The JDK NIO bug produces the situation that empty polling of `Selector` could cause CPU 100%, which refers to
1. [JDK-2147719 : (se) Selector doesn't block on Selector.select(timeout) (lnx)](https://bugs.java.com/bugdatabase/view_bug.do?bug_id=2147719)
2. [JDK-6403933 : (se) Selector doesn't block on Selector.select(timeout) (lnx)](https://bugs.java.com/bugdatabase/view_bug.do?bug_id=6403933)
When the epoll mode is available, the default IO mode should be `EPOLL`, which backports [NettyServer.java#L92](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java#L92). Meanwhile, the transport io mode should be `NIO` when the epoll mode is unavailable.
### Does this PR introduce _any_ user-facing change?
Change the default value of `celeborn.<module>.io.mode`.
### How was this patch tested?
CI.
Closes#2994 from SteNicholas/CELEBORN-1774.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Improve some logs, mainly including checking commit result and waiting partition location empty when worker gracefully shutdown.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
Some logs changed.
### How was this patch tested?
Manual test.
Closes#2995 from onebox-li/improve-logs.
Authored-by: onebox-li <lyh-36@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
1. Add histogram
2. Collect critical metrics about fetch chunk
### Why are the changes needed?
1. To find out IO pattern of fetch chunk
2. To have detail metrics about fetch chunk time
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and cluster.
<img width="940" alt="截屏2024-12-09 15 42 50" src="https://github.com/user-attachments/assets/9f526103-c162-4607-a031-ba90f42ae83e">
<img width="962" alt="截屏2024-12-09 15 42 56" src="https://github.com/user-attachments/assets/c17822da-0433-4701-b0cc-0887ac970353">
Closes#2983 from FMX/b1766.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
This reverts commit b65b5433dc.
<!--
Thanks for sending a pull request! Here are some tips for you:
- Make sure the PR title start w/ a JIRA ticket, e.g. '[CELEBORN-XXXX] Your PR title ...'.
- Be sure to keep the PR description updated to reflect all changes.
- Please write your PR title to summarize what this PR proposes.
- If possible, provide a concise example to reproduce the issue for a faster review.
-->
### What changes were proposed in this pull request?
Revert [CELEBORN-1376](https://github.com/apache/celeborn/pull/2449)
This pr will introduce reference count error when replica enable and workers randomly terminate
### Why are the changes needed?
When data replication is enabled and workers are randomly terminated there will be IllegalReferenceCountException `refCnt: 0, decrement: 1` which will fail the task.

### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
cluster testing.
Closes#2992 from zaynt4606/clbr1376.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Non-IOException (will throw illegalReferenceCountException If a netty's buffer reference count is incorrect)
should also be set in FlushNotifier.
Provides Utils to convert non-IOExceptions to IOExceptions.
### Why are the changes needed?
In some test scenarios where data replication is enabled and workers are randomly terminated, it will throw illegalReferenceCountException which won't be caught.

### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UT & cluster test.
Closes#2988 from zaynt4606/clb1770-m.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix the issue of losing the primary location when parsing `GetReducerFileGroupResponse` from `LifecycleManager`.
### Why are the changes needed?
In previous optimizations, I introduced packed partition locations to reduce the size of RPC calls, based on the assumption that primary partition locations would always be available. However, in some test scenarios where data replication is enabled and workers are randomly terminated, the primary location may be lost while the replica location remains. This causes the replica locations to be ignored which will cause data loss.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and cluster.
Closes#2986 from FMX/b1769.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
As title.
### Why are the changes needed?
https://docs.google.com/document/d/1Jaix22vME0m1Q-JtTHF9WYsrsxBWBwzwmifcPxNQZHk/edit?tab=t.0#heading=h.iadpu3t4rywi
(Thanks to cfmcgrady littlexyw ErikFang waitinfuture RexXiong FMX for their efforts on the proposal)
### Does this PR introduce _any_ user-facing change?
The response of pushMergedData has been modified, however, the changes are backward compatible.
### How was this patch tested?
UT: org.apache.celeborn.service.deploy.cluster.PushMergedDataHardSplitSuite
Closes#2924 from jiang13021/cip-12.
Authored-by: jiang13021 <jiangyanze.jyz@antgroup.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Remove the ConcurrentLinkedQueue and lock in AbstractSource which might cause the metrics data interruption and job fail.
### Why are the changes needed?
Current problems:[jira CELEBORN-1743](https://issues.apache.org/jira/browse/CELEBORN-1743)
the lock in [[CELEBORN-1453]](https://github.com/apache/celeborn/pull/2548) might block the thread.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test
same result with CELEBORN-1453

Closes#2956 from zaynt4606/clb1743.
Authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Deprecate identity configs related with quota –
```
"celeborn.quota.identity.provider"
"celeborn.quota.identity.user-specific.tenant"
"celeborn.quota.identity.user-specific.userName"
```
In favour of identity configs independent of quota
```
"celeborn.identity.provider"
"celeborn.identity.user-specific.tenant"
"celeborn.identity.user-specific.userName"
```
### Why are the changes needed?
Current identity configs are tied with quota but identity should be free of quota because other pieces like tags are also using it. In future other new components can also make use of identity.
### Does this PR introduce _any_ user-facing change?
NA
### How was this patch tested?
Existing UTs
Closes#2952 from s0nskar/fix_identity.
Authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix the worker parses `ReserveSlots` logic for compatibility
### Why are the changes needed?
When upgrading to 0.5, the 0.4 client reserves slots for the 0.5 worker. If there is only a replicate location, the worker parses abnormally, causing the actual reserve to fail, but returns success to the client.
The worker log "Reserved 0 primary location and 0 replica location" appears.
### Does this PR introduce _any_ user-facing change?
When upgrading to 0.5 from 0.4, fix potential reserve slot failure scenario.(only replica location).
### How was this patch tested?
Manual test.
Closes#2968 from onebox-li/fix-reserve-compatibility.
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?
as titile
### Why are the changes needed?
The doc fail to mention S3 as one of storage layers
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Closes#2963 from zhaohehuhu/dev-1128.
Authored-by: zhaohehuhu <luoyedeyi@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
- Adding support to enable/disable worker tags feature by a master config flag.
- Fixed BUG: After this change #2936, admins can also define the tagsExpr for users. In a case user is passing an empty tagsExpr current code will ignore the admin defined tagsExpr and allow job to use all workers.
### Why are the changes needed?
https://cwiki.apache.org/confluence/display/CELEBORN/CIP-11+Supporting+Tags+in+Celeborn
### Does this PR introduce _any_ user-facing change?
NA
### How was this patch tested?
Existing UTs
Closes#2953 from s0nskar/tags-enabled.
Authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Remove the code for app top disk usage both in master and worker end.
Prefer to use below prometheus expr to figure out the top app usages.
```
topk(50, sum by (applicationId) (metrics_diskBytesWritten_Value{role="worker", applicationId!=""}))
```
### Why are the changes needed?
To address comments: https://github.com/apache/celeborn/pull/2947#issuecomment-2499564978
> Due to the application dimension resource consumption, this feature should be included in the deprecated features. Maybe you can remove the codes for application top disk usage.
### Does this PR introduce _any_ user-facing change?
Yes, remove the app top disk usage api.
### How was this patch tested?
GA.
Closes#2949 from turboFei/remove_app_top_usage.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Flink supports fallback to vanilla Flink built-in shuffle implementation.
### Why are the changes needed?
When quota is unenough or workers are unavailable, `RemoteShuffleMaster` does not support fallback to `NettyShuffleMaster`, and `RemoteShuffleEnvironment` does not support fallback to `NettyShuffleEnvironment` at present. Flink should support fallback to vanilla Flink built-in shuffle implementation for unenough quota and unavailable workers.

### Does this PR introduce _any_ user-facing change?
- Introduce `ShuffleFallbackPolicy` interface to determine whether fallback to vanilla Flink built-in shuffle implementation.
```
/**
* The shuffle fallback policy determines whether fallback to vanilla Flink built-in shuffle
* implementation.
*/
public interface ShuffleFallbackPolicy {
/**
* Returns whether fallback to vanilla flink built-in shuffle implementation.
*
* param shuffleContext The job shuffle context of Flink.
* param celebornConf The configuration of Celeborn.
* param lifecycleManager The {link LifecycleManager} of Celeborn.
* return Whether fallback to vanilla flink built-in shuffle implementation.
*/
boolean needFallback(
JobShuffleContext shuffleContext,
CelebornConf celebornConf,
LifecycleManager lifecycleManager);
}
```
- Introduce `celeborn.client.flink.shuffle.fallback.policy` config to support shuffle fallback policy configuration.
### How was this patch tested?
- `RemoteShuffleMasterSuiteJ#testRegisterJobWithForceFallbackPolicy`
- `WordCountTestBase#celeborn flink integration test with fallback - word count`
Closes#2932 from SteNicholas/CELEBORN-1700.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Support predefined tags expression for tenant and users via dynamic config. Using this admin can configure tags for users/tenants and give permission to special users to provide custom tags expression.
### Why are the changes needed?
https://cwiki.apache.org/confluence/display/CELEBORN/CIP-11+Supporting+Tags+in+Celeborn
### Does this PR introduce _any_ user-facing change?
NA
### How was this patch tested?
UTs
Closes#2936 from s0nskar/admin_tags.
Authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
`RpcTimeoutException` adds RPC address in message to help troubleshooting of timeout.
### Why are the changes needed?
The message of `RpcTimeoutException` does not contain the RPC address in the message at present, which causes that the timeout problem is hard to troubleshooting for unknown rpc address.
```
24/11/12 03:00:51 [Executor task launch worker for task 53432.0 in stage 0.0 (TID 53487)] ERROR Executor: Exception in task 53432.0 in stage 0.0 (TID 53487)
org.apache.celeborn.common.rpc.RpcTimeoutException: Futures timed out after [120000 milliseconds]. This timeout is controlled by celeborn.rpc.lookupTimeout
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.RpcEnv.setupEndpointRefByAddr(RpcEnv.scala:106)
at org.apache.celeborn.common.rpc.RpcEnv.setupEndpointRef(RpcEnv.scala:114)
at org.apache.celeborn.client.ShuffleClientImpl.setupLifecycleManagerRef(ShuffleClientImpl.java:1759)
at org.apache.celeborn.client.ShuffleClient.get(ShuffleClient.java:89)
at org.apache.spark.shuffle.celeborn.SparkShuffleManager.getWriter(SparkShuffleManager.java:239)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:57)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:100)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:144)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:598)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1545)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:603)
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)
Caused by: java.util.concurrent.TimeoutException: Futures timed out after [120000 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:316)
at org.apache.celeborn.common.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:74)
... 15 more
```
Therefore, `RpcTimeoutException` should include RPC address in message to help troubleshooting of timeout.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`NettyRpcEnvSuite#ask a message timeout on Future using RpcTimeout`
Closes#2907 from SteNicholas/CELEBORN-1713.
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?
As title
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#2935 from cfmcgrady/CELEBORN-1634-followup.
Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Fu Chen <cfmcgrady@gmail.com>
### What changes were proposed in this pull request?
as title
### Why are the changes needed?
AWS S3 doesn't support append, so Celeborn had to copy the historical data from s3 to worker and write to s3 again, which heavily scales out the write. This PR implements a better solution via MPU to avoid copy-and-write.
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?

I conducted an experiment with a 1GB input dataset to compare the performance of Celeborn using only S3 storage versus using SSD storage. The results showed that Celeborn with SSD storage was approximately three times faster than with only S3 storage.
<img width="1728" alt="Screenshot 2024-11-16 at 13 02 10" src="https://github.com/user-attachments/assets/8f879c47-c01a-4004-9eae-1c266c1f3ef2">
The above screenshot is the second test with 5000 mapper and reducer that I did.
Closes#2830 from zhaohehuhu/dev-1021.
Lead-authored-by: zhaohehuhu <luoyedeyi@163.com>
Co-authored-by: He Zhao <luoyedeyi459@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>