### What changes were proposed in this pull request?
Add release utils tool.
Copied from:
https://github.com/apache/kyuubi/blob/master/build/release/pre_gen_release_notes.pyhttps://github.com/apache/kyuubi/blob/master/build/release/release_utils.py
### Why are the changes needed?
To reduce the release efforts
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
```
RELEASE_TAG=v0.6.0-rc0 PREVIOUS_RELEASE_TAG=v0.5.0 build/release/pre_gen_release_notes.py
```
```
(base) ➜ celeborn-p2 git:(release_utils) RELEASE_TAG=v0.6.0-rc0 PREVIOUS_RELEASE_TAG=v0.5.0 build/release/pre_gen_release_notes.py
Gathering new commits between tags v0.5.0 and v0.6.0-rc0
==================================================================================
Release tag: v0.6.0-rc0
Previous release tag: v0.5.0
Number of commits in this range: 535
Show all commits? [y/n]: y
2d3c48460 Wang, Fei [RELEASE] Bump 0.6.0
f7be34194 Jinqian Fan [CELEBORN-1902] Read client throws PartitionConnectionException (Closes#3147)
2a847ba90 Wang, Fei [MINOR] Change some config version (Closes#3269)
...
142d0a053 mingji Bump 0.6.0-SNAPSHOT
==================================================================================
Does this look correct? [y/n]: y
==================================================================================
Found 1 release commits
Found 2 revert commits
Found 19 commits with no Ticket
==================== Warning: these commits will be ignored ======================
Release (1)
2d3c48460 Wang, Fei [RELEASE] Bump 0.6.0
Revert (2)
c316fdbdf zaynt4606 Revert "[CELEBORN-1376] Push data failed should always release request body" (Closes#2992) (Reverts b65b5433d)
8d0b4cf4c waitinfuture [CELEBORN-1506][BUG] Revert "[CELEBORN-1036][FOLLOWUP] totalInflightReqs should decrement when batchIdSet contains the batchId to avoid duplicate caller of removeBatch" (Closes#2621)
No Ticket (19)
2a847ba90 Wang, Fei [MINOR] Change some config version (Closes#3269)
54732c7b3 Nicolas Fraison Update celeborn conf to add S3 in default and doc for policy (Closes#3218)
a06362259 Cheng Pan [MINOR][INFRA] Do not cancel GHA jobs on committing to main/branch-* branches (Closes#3235)
529fd6e01 cxzl25 [MINOR] Avoid use `_$eq` in Scala file (Closes#3208)
dfeaef135 Cheng Pan [MINOR] Add spec link to JavaSerializer (Closes#3194)
05b6ad4a7 Sanskar Modi [MINOR] Change config versions (Closes#3142)
6f5ad2dde Wang, Fei [MINOR] Refine the log for fetch failure and rpc metrics dump (Closes#3136)
b9e4bbb5a cxzl25 [MINOR] Change some config version (Closes#3082)
4ccb0c7fc SteNicholas [MINOR] Rename org.apache.celeborn.plugin.flink.readclient to org.apache.celeborn.plugin.flink.client (Closes#3048)
80523214e Sanskar Modi [MINOR] Add documentation for `CELEBORN_NO_DAEMONIZE` (Closes#3020)
43e1b8a24 FMX [MINOR] Update DingTalk group link (Closes#2948)
7fbf0e2fa Wang, Fei [MINOR] Fix missing blanks in docs (Closes#2917)
71e3c03a1 Wang, Fei [MINOR] Fix docs typo (Closes#2890)
d44b23c85 Weijie Guo [MINOR] Remove unused TODO comments in CelebornTierProducerAgent#processBuffer (Closes#2883)
7018996e2 jiang13021 [MINOR] Fix typo in ExceptionUtils (Closes#2841)
8bd5ac0b9 SteNicholas [MINOR] Add navigation for REST API document (Closes#2775)
3cc043a17 SteNicholas [MINOR] Delete DEPLOY_ON_K8S.md (Closes#2752)
f226424b9 Bowen Liang [CLEBORN-1555] Replace deprecated config celeborn.storage.activeTypes in docs and tests (Closes#2675)
142d0a053 mingji Bump 0.6.0-SNAPSHOT
==================== Warning: the above commits will be ignored ==================
513 effective commits left to process after filtering. OK to proceed? [y/n]: y
=========================== Compiling contributor list ===========================
Processed commit f7be34194 authored by Jinqian Fan on Wed May 21 16:58:30 2025 -0700
Processed commit 082f0dd8c authored by Sanskar Modi on Wed May 21 16:37:38 2025 -0700
Processed commit 45b94bf05 authored by Yi Chen on Wed May 21 01:21:45 2025 -0700
...
Processed commit 450dac824 authored by Nicholas Jiang on Mon Jun 3 17:47:01 2024 +0800
==================================================================================
Commits list is successfully written to commits-v0.6.0-rc0.txt!
Contributors list is successfully written to contributors-v0.6.0-rc0.txt!
============ Warnings encountered while creating the contributor list ============
Found the following invalid authors:
avishnus
Madhukar525722
xy2953396112
Please update 'known_translations'.
Please correct these in the final contributors list at contributors-v0.6.0-rc0.txt.
==================================================================================
```
```
cat build/release/contributors-v0.6.0-rc0.txt
* Tao Zheng
* Amandeep Singh
* Ziyi Wu
* Sanskar Modi
* Yuting Wang
* Cheng Pan
* Aravind Patnam
* Zhao Zhao
* Saurabh Dubey
* xy2953396112
* Bowen Liang
* Shlomi Uubul
* Erik Fang
* Yi Chen
* Leo Li
* Nicolas Fraison
* Jiashu Xiong
* Pengqi Li
* Jiaming Xie
* Keyong Zhou
* Jinqian Fan
* Guangwei Hong
* Yi Zhu
* Madhukar525722
* Jianfu Li
* Chongchen Chen
* Biao Geng
* Lianne Li
* Fei Wang
* Mridul Muralidharan
* Wang, Fei
* avishnus
* Xu Huang
* Weijie Guo
* Xinyu Wang
* Yajun Gao
* He Zhao
* Björn Boschman
* Shaoyun Chen
* Kerwin Zhang
* Kun Wan
* Zhengqi Zhang
* Minchu Yang
* Haotian Cao
* Xianming Lei
* Shengjie Wang
* Veli Yang
* Arsen Gumin
* Mingxiao Feng
* Yuxin Tan
* Aidar Bariev
* Nan Zhu
* Fu Chen
* Binjie Yang
* Yanze Jiang
* Nicholas Jiang
```
```
cat build/release/commits-v0.6.0-rc0.txt|wc -l
532
```
Closes#3280 from turboFei/release_utils.
Authored-by: Fei Wang <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Use `tmp` subfolder for svc staging dir.
### Why are the changes needed?
Refer:
81c3d91f75/build/release/release.sh (L67)
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Local.
Closes#3278 from turboFei/release_guide_follow.
Authored-by: Fei Wang <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Add release guide and fix several issues during 0.6.0 release.
### Why are the changes needed?
Add docs.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Tested locally.
Closes#3271 from turboFei/release_guide.
Authored-by: Fei Wang <fwang12@ebay.com>
Signed-off-by: Fei Wang <fwang12@ebay.com>
### What changes were proposed in this pull request?
Introduce `numBytesIn`, `numBytesOut`, `numRecordsOut`, `numBytesInPerSecond`, `numBytesOutPerSecond`, `numRecordsOutPerSecond` metrics for `RemoteShuffleServiceFactory`.
Scope | Infix | Metrics | Description | Type
-- | -- | -- | -- | --
Task | Shuffle.Remote.[ShuffleId] | numBytesIn | The total number of bytes this shuffle has read. | Counter |
Task | Shuffle.Remote.[ShuffleId] | numBytesOut | The total number of bytes this shuffle has written. | Counter |
Task | Shuffle.Remote.[ShuffleId] | numRecordsOut | The total number of records this shuffle has written. | Counter |
Task | Shuffle.Remote.[ShuffleId] | numBytesInPerSecond | The number of bytes this shuffle reads per second. | Meter |
Task | Shuffle.Remote.[ShuffleId] | numBytesOutPerSecond | The number of bytes this shuffle writes per second. | Meter |
Task | Shuffle.Remote.[ShuffleId] | numRecordsOutPerSecond | The number of records this shuffle writes per second. | Meter |
Note:
- `numBytesIn` and `numBytesOut` metrics include the total number of bytes for records and events.
- `numRecordsOut` metric only includes the total number of records, instead of records and events.
### Why are the changes needed?
There is no any metrics related to shuffle read operations and operations writing shuffle data for flink shuffle. It's proposed to introduce `numBytesIn`, `numBytesOut`, `numRecordsOut`, `numBytesInPerSecond`, `numBytesOutPerSecond`, `numRecordsOutPerSecond` metrics for `RemoteShuffleServiceFactory`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `RemoteShuffleOutputGateSuiteJ#testSimpleWriteData`
- `RemoteShuffleResultPartitionSuiteJ`
Closes#3272 from SteNicholas/CELEBORN-2005.
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?
`org.apache.celeborn.client.LifecycleManager.getPartitionType` may be called frequently, but in Spark scenario it requires each parsing configuration, which is not necessary.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#3273 from cxzl25/CELEBORN-2006.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
Driver may have a large number of `PartitionLocation` objects, reducing some unnecessary fields of `PartitionLocation` can reduce the memory pressure of Driver.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#3274 from cxzl25/CELEBORN-2007.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
SlotsAllocator should select disks randomly in RoundRobin mode
### Why are the changes needed?
The current round robin selection mechanism is to select the first disk of each worker first, then the second disk of each worker, and finally the third disk. This can easily cause disk storage space skew. We should select disks randomly instead of selecting the first disk first.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#3275 from leixm/CELEBORN-2008.
Authored-by: Xianming Lei <xianming.lei@shopee.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
`org.apache.flink.runtime.io.network.partition.consumer.PartitionConnectionException` is thrown when RemoteBufferStreamReader finds that the current exception is about connection failure.
### Why are the changes needed?
If `org.apache.flink.runtime.io.network.partition.consumer.PartitionConnectionException` is correctly thrown to reflect connection failure, then Flink can be aware of the lost Celeborn server side nodes and be able to re-compute affected data. Otherwise, endless retries could cause Flink job failure.
This PR is to deal with exceptions like:
```
java.io.IOException: org.apache.celeborn.common.exception.CelebornIOException: Failed to connect to ltx1-app10154.prod.linkedin.com/10.88.105.20:23924
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Tested in a Flink batch job with Celeborn.
Closes#3147 from Austinfjq/throw-Partition-Connection-Exception.
Lead-authored-by: Jinqian Fan <jinqianfan@icloud.com>
Co-authored-by: Austin Fan <jinqianfan@icloud.com>
Co-authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Fix the false config version in https://celeborn.apache.org/docs/0.5.4/configuration/
In https://github.com/apache/celeborn/pull/3082, it fixed:
- celeborn.master.endpoints.resolver
- celeborn.client.chunk.prefetch.enabled
- celeborn.client.inputStream.creation.window
In this PR, it fixes the remaining
- celeborn.ssl.<module>.sslHandshakeTimeoutMs
### Why are the changes needed?
Fix the false config version in https://celeborn.apache.org/docs/0.5.4/configuration/
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
GA
Closes#3269 from turboFei/config_version.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Minor logging improvement around commit files to log shuffleKey.
### Why are the changes needed?
Ditto.
### Does this PR introduce _any_ user-facing change?
Some logs will change.
### How was this patch tested?
NA
Closes#3270 from s0nskar/CELEBORN-1775.
Authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
- Rename `volumes.master` to `master.volumes` and `master.volumeMounts`.
- Rename `volumes.worker` to `worker.volumes` and `worker.volumeMounts`.
- Users will need to configure `celeborn.master.ha.ratis.raft.server.storage.dir` manually.
- Users will need to configure `celeborn.worker.storage.dirs` manually.
### Why are the changes needed?
- Unify the values naming by prefixing them with `master` or `worker`.
- Provide users the maximum flexibility to configure storage.
- It will be easier to implement persistentVolumeClaims feature.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
Run Helm unit tests by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3254 from ChenYi015/helm/volumes.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Improve Aliyun OSS support including `SlotsAllocator#offerSlotsLoadAware`, `Worker#heartbeatToMaster` and `PartitionDataWriter#getStorageInfo`.
### Why are the changes needed?
There are many methods where OSS support is lacking in `SlotsAllocator#offerSlotsLoadAware`, `Worker#heartbeatToMaster` and `PartitionDataWriter#getStorageInfo`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3268 from SteNicholas/CELEBORN-1916.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
it's a joint work with YutingWang98
currently we have to wait for spark shuffle object gc to clean disk space occupied by celeborn shuffles
As a result, if a shuffle is failed to fetch and retried , the disk space occupied by the failed attempt cannot really be cleaned , we hit this issue internally when we have to deal with 100s of TB level shuffles in a single spark application, any hiccup in servers can double even triple the disk usage
this PR implements the mechanism to delete files from failed-to-fetch shuffles
the main idea is actually simple, it triggers clean up in LifecycleManager when it applies for a new celeborn shuffle id for a retried shuffle write stage
the tricky part is that is to avoid delete shuffle files when it is referred by multiple downstream stages: the PR introduces RunningStageManager to track the dependency between stages
### Why are the changes needed?
saving disk space
### Does this PR introduce _any_ user-facing change?
a new config
### How was this patch tested?
we manually delete some files

from the above screenshot we can see that originally we have shuffle 0, 1 and after 1 faced with chunk fetch failure, it triggers a retry of 0 (shuffle 2), but at this moment, 0 has been deleted from the workers

in the logs, we can see that in the middle the application , the unregister shuffle request was sent for shuffle 0
Closes#3109 from CodingCat/delete_fi.
Lead-authored-by: CodingCat <zhunansjtu@gmail.com>
Co-authored-by: Wang, Fei <fwang12@ebay.com>
Co-authored-by: Fei Wang <cn.feiwang@gmail.com>
Co-authored-by: Fei Wang <fwang12@ebay.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Filter empty partition from partitionFileGroup before createIntputStream.
### Why are the changes needed?
Avoid creating the IntputStream for the empty partitions which might be a lot when partition num is large and data is small.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
UT and cluster test.
Closes#3266 from zaynt4606/clb2004.
Lead-authored-by: Fei Wang <cn.feiwang@gmail.com>
Co-authored-by: zhengtao <shuaizhentao.szt@alibaba-inc.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
`CelebornConf` introduces `celeborn.<module>.io.threads` to specify number of threads used in the client thread pool.
### Why are the changes needed?
`ShuffleClientImpl` and `FlinkShuffleClientImpl` use fixed configuration expression as `conf.getInt("celeborn." + module + ".io.threads", 8)`. Therefore, `CelebornConf` should introduce `celeborn.<module>.io.threads` to specify number of threads used in the client thread pool.
### Does this PR introduce _any_ user-facing change?
`CelebornConf` adds `celeborn.<module>.io.threads` config option.
### How was this patch tested?
No.
Closes#3245 from SteNicholas/CELEBORN-1993.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Ignore the getReducerFileGroup timeout before shuffle stage end.
### Why are the changes needed?
1. if the getReducerFileGroup timeout is caused by lifecycle manager commitFiles timeout(stage not ended)
2. maybe many tasks failed and would not report fetch failure
3. then it cause the spark application failed eventually.
The shuffle client should ignore the getReducerFileGroup timeout before LifeCycleManager commitFiles complete.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
UT
Closes#3263 from turboFei/is_stage_end.
Lead-authored-by: Wang, Fei <fwang12@ebay.com>
Co-authored-by: Fei Wang <cn.feiwang@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Introduce `ApplicationTotalCount` and `ApplicationFallbackCount` metric to record the total and fallback count of application.
### Why are the changes needed?
There is no any metric to record the total count of application running with celeborn shuffle and engine bulit-in shuffle and the fallback count of application. Meanwhile, the fallback of Flink shuffle is based on job granularity rather than shuffle granularity.
Follw up https://github.com/apache/celeborn/pull/3012#issuecomment-2553488532.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
- `DefaultMetaSystemSuiteJ#testShuffleAndApplicationCountWithFallback`
- `RatisMasterStatusSystemSuiteJ#testShuffleAndApplicationCountWithFallback`
Closes#3026 from SteNicholas/CELEBORN-1800.
Lead-authored-by: SteNicholas <programgeek@163.com>
Co-authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Aggregate push failed batch for the same map ID and attempt ID.
### Why are the changes needed?
To reduce memory usage.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and cluster run.
Closes#3253 from FMX/b1995.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Always log the size of GetReducerFileGroupResponse, which is the most heaviest RPC in client end.
### Why are the changes needed?
To provide more insights if meet rpc timeout for GetReducerFileGroupResponse.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
GA.
Closes#3262 from turboFei/broadcast_size.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix the issue that upstream tasks don't rerun and the current task still retry when failed to deserialize in flink. `RemoteShuffleInputGateDelegation` should keep the same behavior when failed to decompress.
Follow up #2884.
### Why are the changes needed?
Deserialize error should retry upstream otherwise this is un-recoverable.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual.
Closes#3255 from SteNicholas/CELEBORN-1691.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
`RemoteShuffleEnvironment` should not register `InputChannelMetrics` repeatedly, which only create once in `RemoteShuffleEnvironment#createInputGates`.
### Why are the changes needed?
If a task has multiple input gates, It will create as many `InputChannelMetrics` as the number of gates, so the corresponding metrics are registered repeatedly. Therefore, `RemoteShuffleEnvironment` should not register `InputChannelMetrics` repeatedly for `createInputGates`.
Backport https://github.com/apache/flink/pull/21437.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual.
Closes#3257 from SteNicholas/CELEBORN-1998.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
OpenStreamTime should use requestId to record cost time instead of shuffleKey
### Why are the changes needed?
OpenStreamTime is wrong because there will be multiple OpenStream requests for the same shuffleKey in the same time period.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#3258 from leixm/CELEBORN-1999.
Authored-by: Xianming Lei <xianming.lei@shopee.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Introduce disruptor dependency to support asynchronous logging of log4j2.
### Why are the changes needed?
We add `-Dlog4j2.contextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector` in `CELEBORN_MASTER_JAVA_OPTS` and `CELEBORN_WOKRER_JAVA_OPTS` for production environment. `AsyncLoggerContextSelector` depends on disruptor dependency. Therefore, it's recommend to introduce disruptor dependency to support log4j2 asynchronous loggers.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Cluster test.
Closes#3246 from SteNicholas/CELEBORN-1994.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
- Split `securityContext` into `master.podSecurityContext` and `worker.podSecurityContext`.
- Add `master.securityContext` and `worker.securityContext` for container-level security configurations.
### Why are the changes needed?
Allow separate configurations for master/worker pod-level/container-level security context.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
Run Helm unit tests by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3251 from ChenYi015/helm/security-context.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
for non barrier shuffle read stage, LifecycleManager#handleGetShuffleIdForApp always return appshuffleId whether fetch status is true or not.
### Why are the changes needed?
As described in [jira](https://issues.apache.org/jira/browse/CELEBORN-1855), If LifecycleManager only returns appshuffleId whose fetch status is success, the task will fail directly to "there is no finished map stage associated with", but previous fetch fail event reported may not be fatal.So just give it a chance
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
Closes#3090 from buska88/celeborn-1855.
Authored-by: lijianfu03 <lijianfu@meituan.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Push applicationId as metrics label only if `celeborn.metrics.worker.appLevel.enabled` is true.
### Why are the changes needed?
At Uber, We use m3 for monitoring, it tries to make a new series using all the present metrics label. Having applicationId as a metrics introduces too much cardinality in `activeconnectioncount` and we are unable to use it, while it is an useful metric with/without applicationId as label. Similarly for resourceConsumption, userIdentifier alone can be used.
### Does this PR introduce _any_ user-facing change?
Yes, changed the default config value.
### How was this patch tested?
NA
Closes#3221 from s0nskar/application_tag.
Lead-authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Co-authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Adding a worker metrics for publish unreleased partition location count when worker was gracefully shutdown.
<img width="742" alt="Screenshot 2025-04-16 at 1 19 18 AM" src="https://github.com/user-attachments/assets/159f744a-cd76-45a2-9387-930f27dd72be" />
### Why are the changes needed?
Similar to https://github.com/apache/celeborn/pull/2711, Currently celeborn don't publish the count of unreleased partition location when worker is gracefully exit. This can be useful for monitoring and configuring the gracefulShutdownTimeout.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
NA
Closes#3213 from s0nskar/unrelease_partition_location.
Lead-authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Co-authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
This PR adds code style checking in github action for cppClient.
### Why are the changes needed?
To keep the cpp code style consistent.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
By github action procedure.
Closes#3252 from HolyLow/issue/celeborn-1978-add-codestyle-check-to-cppclient.
Authored-by: HolyLow <jiaming.xie7@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
In the `updateProduceBytes` method, `userBufferInfo.bufferStatusHub` and `workerBufferStatusHub` add the same node. This results in the lastNode's numBytes being repeatedly accumulated in the `_deque` of both hubs if updateProduceBytes is called multiple times within one second, since the `lastNode` is the same object.
This will cause the numBytes of `nodeToSeparate` to become negative when the `removeExpiredNodes` method is called, specifically at the line `nodeToSeparate.separateNode(removed.getRight());` , which will make avgBytesPerSec return 0.
As shown in the figure below, this value remains at 0 for a long period of time in the Dashborad.

### Why are the changes needed?
Fix updateProduceBytes method.
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
UT
AND Dashborad:

Closes#3240 from vastian180/CELEBORN-1487-FOLLOWUP.
Authored-by: caohaotian <caohaotian@meituan.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Support all [default hadoop provider](https://github.com/apache/hadoop/blob/rel/release-3.3.6/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java#L563) for S3 authentication
### Why are the changes needed?
As of now celeborn only support authentication based on ACESS/SECRET key while other authentication mechanism can be required (for ex. ENV var, relying on [AWS_CONTAINER_CREDENTIALS_RELATIVE_URI](https://docs.aws.amazon.com/sdkref/latest/guide/feature-container-credentials.html))
### Does this PR introduce _any_ user-facing change?
yes, the `celeborn.storage.s3.secret.key` and `celeborn.storage.s3.access.key` are removed. In order to still provide those we should rely on the hadoop config (`celeborn.hadoop.fs.s3a.access.key` / `celeborn.hadoop.fs.s3a.secret.key `)
### How was this patch tested?
Tested on celeborn cluster deployed on kubernetes and configured to use S3 relying on `IAMInstanceCredentialsProvider`
Closes#3243 from ashangit/nfraison/CELEBORN-1965.
Lead-authored-by: Nicolas Fraison <nfraison@yahoo.fr>
Co-authored-by: nicolas.fraison@datadoghq.com <nicolas.fraison@datadoghq.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
- Split `hostNetwork` into `master.hostNetwork` and `worker.hostNetwork`.
### Why are the changes needed?
Unify the values naming by prefixing them with `master` or `worker`.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
Run Helm unit tests by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3250 from ChenYi015/helm/host-network.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix data lost when push merged data of replica and hard split happen.
### Why are the changes needed?
There is a problem with replicate rpc callback. The code should satisfy the following conditions: when comparing the status returned by primary and replica data, the status on the left should be used as the final status for the client, FAILURE > HARD_SPLIT > CONGESTION > SUCCESS. The status on the right cannot cover the status on the left.
There are two problems with the code now
1. HARD_SPLIT can cover FAILURE, which will affect the exclude worker logic, and there may be some problems
2. When processing a pushMergedData request, some partitionLocations are committed, PushDataHandler cannot stop pushing replicas as long as there are any partitions that have not been committed, otherwise data loss will occur.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#3185 from leixm/CELEBORN-1948.
Lead-authored-by: Xianming Lei <31424839+leixm@users.noreply.github.com>
Co-authored-by: Xianming Lei <xianming.lei@shopee.com>
Co-authored-by: Shuang <lvshuang.xjs@alibaba-inc.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
- Split `dnsPolicy` into `master.dnsPolicy` and `worker.dnsPolicy`
### Why are the changes needed?
Unify the values naming by prefixing them with `master` or `worker`.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
Run Helm unit tests by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3236 from ChenYi015/helm/dns-policy.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
`PushState ` uses `JavaUtils#newConcurrentHashMap` to speed up `ConcurrentHashMap#computeIfAbsent`.
### Why are the changes needed?
Celeborn supports JDK8, which could meet the bug mentioned in [JDK-8161372](https://bugs.openjdk.org/browse/JDK-8161372). Therefore, it's better to use `JavaUtils#newConcurrentHashMap` to speed up `ConcurrentHashMap#computeIfAbsent`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3247 from SteNicholas/CELEBORN-1319.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Remove the redundant release of data after OutOfDirectMemoryError appears in flushBuffer.addComponent
### Why are the changes needed?
The reason why OutOfDirectMemoryError will appear in flushBuffer.addComponent is that after adding a new component, CompositeByteBuf will determine whether the number of components exceeds the maximum limit. If it exceeds, the existing components will be merged into a large component. At this time, new off-heap memory will be requested. If there is insufficient memory at this time, OutOfDirectMemoryError will be reported, but the new component has been added to flushBuffer at this time. Releasing the new component at this time will cause refcnt error.
Don't worry about the component here not being released causing memory leaks, because it will be released normally in returnBuffer (flush or file destroy or file close).
If writeLocalData does not catch OutOfDirectMemoryError, the impact is as follows:
1. In the case of a single copy, if https://github.com/apache/celeborn/pull/3049 pr is not merged, commitfile will be blocked in waitPendingWrites and fail, because writeLocalData does not correctly decrementPendingWrites. However, this will not cause flushBuffer to exist in memory for a long time, because when shuffle expires, the file will be destroyed, flushBuffer will be returned, and this part of memory will be released.
2. In the case of dual replicas, in addition to the above problems, the thread of the Eventloop to which replicate-client belongs will be blocked at Await.result(writePromise.future, Duration.Inf) because writePromise is not closed correctly. As a result, this thread will not process other PushData data written by worker-data-replicator to the channels of the Eventloop to which replicate-client belongs. This part of data accumulates in the taskQueue of EventLoop and cannot be canceled, which is the cause of memory leak.
<img width="1081" alt="image" src="https://github.com/user-attachments/assets/a90ac423-443e-42f9-a0d2-cc49f24f6476" />
Therefore, if the memory leak occurs after OutOfDirectMemoryError occurs in flushBuffer.addComponent, you only need to catch OutOfDirectMemoryError in writeLocalData, and there is no need to release data after addComponent.
I simulated the scenario where addCompoent had an OutOfDirectMemoryError, and released data after the OutOfDirectMemoryError occurred, and a refcnt error occurred.
[oom_fix_error_release.log](https://github.com/user-attachments/files/19863484/oom_fix_error_release.log)
At the same time, I simulated the scenario where addCompoent had an OutOfDirectMemoryError and did not release data after the OutOfDirectMemoryError occurred. No refcnt error occurred, commitfiles succeeded, the spark task succeeded, and after commitfiles, the worker diskbuffercount became 0.
[celeborn_1760_followup_worker.log](https://github.com/user-attachments/files/19864486/celeborn_1760_followup_worker.log)
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
manual test.
Closes#3224 from littlexyw/CELEBORN-1760-FOLLOWUP.
Authored-by: xinyuwang1 <xinyuwang1@xiaohongshu.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Add S3 type in evict and create policies
Add S3 type in list of default evict and create policy
### Why are the changes needed?
To align with other types
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Closes#3218 from ashangit/nfraison/doc_s3.
Authored-by: Nicolas Fraison <nfraison@yahoo.fr>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Client should send heartbeat to worker for processing heartbeat to avoid reading idleness of worker which enables heartbeat.
Follow up #1457.
### Why are the changes needed?
In Flink batch jobs, the following exception is caused by closed connection:
```
2025-04-27 23:30:28
java.io.IOException: Client /:9093 is lost, notify related stream 805472050177
at org.apache.celeborn.plugin.flink.RemoteBufferStreamReader.errorReceived(RemoteBufferStreamReader.java:146)
at org.apache.celeborn.plugin.flink.RemoteBufferStreamReader.lambda$new$0(RemoteBufferStreamReader.java:77)
at org.apache.celeborn.plugin.flink.network.ReadClientHandler.processMessageInternal(ReadClientHandler.java:64)
at org.apache.celeborn.plugin.flink.network.ReadClientHandler.lambda$channelInactive$0(ReadClientHandler.java:145)
at java.base/java.util.concurrent.ConcurrentHashMap.forEach(ConcurrentHashMap.java:1603)
at org.apache.celeborn.plugin.flink.network.ReadClientHandler.channelInactive(ReadClientHandler.java:136)
at org.apache.celeborn.common.network.server.TransportRequestHandler.channelInactive(TransportRequestHandler.java:74)
at org.apache.celeborn.common.network.server.TransportChannelHandler.channelInactive(TransportChannelHandler.java:141)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:303)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
at org.apache.celeborn.shaded.io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
at org.apache.celeborn.common.network.client.ReconnectHandler.scheduleReconnect(ReconnectHandler.java:93)
at org.apache.celeborn.common.network.client.ReconnectHandler.channelInactive(ReconnectHandler.java:63)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:303)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
at org.apache.celeborn.shaded.io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
at org.apache.celeborn.shaded.io.netty.handler.timeout.IdleStateHandler.channelInactive(IdleStateHandler.java:280)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:303)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
at org.apache.celeborn.shaded.io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:81)
at org.apache.celeborn.plugin.flink.network.TransportFrameDecoderWithBufferSupplier.channelInactive(TransportFrameDecoderWithBufferSupplier.java:207)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:303)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:274)
at org.apache.celeborn.shaded.io.netty.channel.DefaultChannelPipeline$HeadContext.channelInactive(DefaultChannelPipeline.java:1405)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:301)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:281)
at org.apache.celeborn.shaded.io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:901)
at org.apache.celeborn.shaded.io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:813)
at org.apache.celeborn.shaded.io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173)
at org.apache.celeborn.shaded.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166)
at org.apache.celeborn.shaded.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470)
at org.apache.celeborn.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:566)
at org.apache.celeborn.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at org.apache.celeborn.shaded.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at org.apache.celeborn.shaded.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:991)
```
The closed connection is caused by reading idleness of worker which enables heartbeat with troubleshooting via debug mode of log.
```
2025-04-27 23:30:32,341 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] WRITE: MessageWithHeader [headerLength: 17, bodyLength: 26]
2025-04-27 23:30:32,341 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] FLUSH
2025-04-27 23:30:32,380 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] READ 38B
2025-04-27 23:30:32,380 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] READ COMPLETE
2025-04-27 23:31:31,813 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] WRITE 10B
2025-04-27 23:31:31,813 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] FLUSH
2025-04-27 23:32:31,823 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] WRITE 10B
2025-04-27 23:32:31,824 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] FLUSH
2025-04-27 23:33:31,826 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] WRITE 10B
2025-04-27 23:33:31,826 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] FLUSH
2025-04-27 23:34:31,826 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] WRITE 10B
2025-04-27 23:34:31,826 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] FLUSH
2025-04-27 23:34:32,380 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 - R:/33.133.79.187:44862] CLOSE
2025-04-27 23:34:32,380 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 ! R:/33.133.79.187:44862] INACTIVE
2025-04-27 23:34:32,380 [fetch-server-11-5] DEBUG util.NettyLogger - [id: 0x2dc85987, L:/:9093 ! R:/33.133.79.187:44862] UNREGISTERED
```
The reading idleness of worker which enables heartbeat is resulted via one-way heartbeat from worker to client, which only keeps the channel of client active. Client should handle heartbeat to keep the channel of worker active via sending heartbeat to worker.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`HeartbeatTest`
Closes#3239 from SteNicholas/CELEBORN-1912.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Ensure hadoop FS are not closed by hadoop ShutdownHookManager
### Why are the changes needed?
By default hadoop manage close of the hadoop FS through[ShutdownHookManager](b4466a3b0a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java)
This can leads to having the FS being closed before having all streams being closed
This is leading to issue with S3 which try to perform some call from the s3 hadoop FS to generate index file
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
- Tested on a celeborn cluster installed on kubernetes
- launched a 10 TiB shuffle jobs
- restart some workers while the shuffle job is running
- the files are now well completed and we are not seeing anymore failure on jobs when reading the shuffle data due to missing index files. Also on S3 we do not see anymore some files not completed (data files at 0B)
Closes#3241 from ashangit/nfraison/CELEBORN-1992.
Authored-by: nicolas.fraison@datadoghq.com <nicolas.fraison@datadoghq.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Add bytes written threshold for top app consumption metrics.
### Why are the changes needed?
Used to limit and reduce the top app consumption metrics.
### Does this PR introduce _any_ user-facing change?
New config.
### How was this patch tested?
Existing GA.
Closes#3232 from turboFei/top_resource_consump_threashold.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
- Rename `masterReplicas` to `master.replicas`.
- Rename `workerReplicas` to `worker.replicas`.
### Why are the changes needed?
Unify the values naming by prefixing them with `master` or `worker`.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
`helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3227 from ChenYi015/helm/replicas.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
As the title.
### Why are the changes needed?
To avoid canceling GHA jobs after quickly merging more than one PRs in a short time.
<img width="867" alt="image" src="https://github.com/user-attachments/assets/fe34ee02-3dd5-4b10-89f2-3826031d1bf4" />
### Does this PR introduce _any_ user-facing change?
No, dev-only.
### How was this patch tested?
Monitor after merging.
Closes#3235 from pan3793/minor-cancel-ci.
Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
- Add new value `master.envFrom`
- Add new value `worker.envFrom`
### Why are the changes needed?
Users are allowed to populate container environment variables from ConfigMap/Secret sources.
### Does this PR introduce _any_ user-facing change?
Yes, new values are added.
### How was this patch tested?
Run Helm unit tests by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3233 from ChenYi015/helm/env-from.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
- Rename `priorityClass.master` to `master.priorityClass`
- Rename `priorityClass.worker` to `worker.priorityClass`
### Why are the changes needed?
Unify the values naming by prefixing them with `master` or `worker`.
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
Run Helm unit tests by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3234 from ChenYi015/helm/priority-class.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Add help/type on prometheus exposed metrics:
```
# HELP metrics_UpdateResourceConsumptionTime_Count
# TYPE metrics_UpdateResourceConsumptionTime_Count counter
metrics_UpdateResourceConsumptionTime_Count{instance="192.168.192.143:9098",role="master"} 1 1745390288743
```
### Why are the changes needed?
Datadog agent rely on this type to discover the type of the exposed prometheus metrics: https://docs.datadoghq.com/integrations/openmetrics/#missing-untyped-metrics
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Started one master and worker celeborn instance with below metrics.properties config:
```
*.sink.prometheusServlet.class=org.apache.celeborn.common.metrics.sink.PrometheusServlet
*.sink.jsonServlet.class=org.apache.celeborn.common.metrics.sink.JsonServlet
```
Then connected to the master and worker metrics endpoint for prometheus.
All the metrics now have the help/type annotation.
Closes#3223 from ashangit/nfraison/prometheus.
Authored-by: Nicolas Fraison <nfraison@yahoo.fr>
Signed-off-by: Cheng Pan <chengpan@apache.org>
### What changes were proposed in this pull request?
Remove unused batch data tracking
### Why are the changes needed?
When the optimization to handle skewed partition reads is enabled, Celeborn typically tracks all failed batches to avoid potential data duplication. However, Tracking of hardsplit batch can be safely disabled when pushing a single replica, as data never write to partition data file. as these batches would definitively not write to their previous partition locations. Therefore, Celeborn does not need to track these batches, as doing so could overload the Driver
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manual test & Pass GA
Closes#3164 from RexXiong/CELEBORN-1919.
Authored-by: Shuang <lvshuang.xjs@alibaba-inc.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Change partition manager relocates partition locations will respect the value of `celeborn.storage.availableTypes`.
### Why are the changes needed?
In the current implementation, partition locations created by the change partition manager will use all available storage tiers, which is unexpected.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA.
Closes#3229 from FMX/b1979.
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?
Split `environments` into `master.env` and `worker.env`
### Why are the changes needed?
- Users are able to configure environment variables for master and worker separately.
- Users can use environment variable source to configure envs( e.g. read env from a ConfigMap or Secret).
### Does this PR introduce _any_ user-facing change?
Yes.
### How was this patch tested?
Helm unit test by `helm unittest charts/celeborn --file "tests/**/*_test.yaml" --strict --debug`.
Closes#3226 from ChenYi015/refactor-helm/env.
Authored-by: Yi Chen <github@chenyicn.net>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>