### What changes were proposed in this pull request?
Fix NPE if `celeborn.client.spark.fetch.cleanFailedShuffle` is true.
This PR also refine the code for `FailedShuffleCleaner`.
### Why are the changes needed?
`failedShuffleCleaner` is null in executor end.
```
25/07/29 17:58:40 ERROR SparkUncaughtExceptionHandler: Uncaught exception in thread Thread[CoarseGrainedExecutorBackend-stop-executor,5,main]
java.lang.NullPointerException: Cannot invoke "org.apache.celeborn.spark.FailedShuffleCleaner.reset()" because "this.failedShuffleCleaner" is null
at org.apache.spark.shuffle.celeborn.SparkShuffleManager.stop(SparkShuffleManager.java:272) ~[celeborn-client-spark-3-shaded_2.12-0.6.0-rc3.jar:?]
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
UT.
Closes#3401 from turboFei/fix_npe_cleaner.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Inc COMMIT_FILES_FAIL_COUNT when TimerWriter::close timeout
### Why are the changes needed?
1. the COMMIT_FILES_FAIL_COUNT is 0 even we meet SHUFFLE_DATA_LOST caused by commit files failure
Spark executor log:
```
25/07/30 10:10:39 WARN CelebornShuffleReader: Handle fetch exceptions for 0-0org.apache.celeborn.common.exception.CelebornIOException: Failed to load file group of shuffle 0 partition 441! Request GetReducerFileGroup(0,false,V1) return SHUFFLE_DATA_LOST for 0.
```
Spark driver log:
```
25/07/30 10:10:38 ERROR ReducePartitionCommitHandler: Failed to handle stageEnd for 0, lost file!
25/07/30 10:10:38 ERROR ReducePartitionCommitHandler:
For shuffle application_1750652300305_10219240_1-0 partition data lost:
Lost partition 307-0 in worker [Host:hdc42-mcc10-01-0910-2704-064-tess0028.stratus.rno.ebay.com:RpcPort:9200:PushPort:9202:FetchPort:9201:ReplicatePort:9203]
Lost partition 1289-0 in worker [Host:hdc42-mcc10-01-0910-2704-064-tess0028.stratus.rno.ebay.com:RpcPort:9200:PushPort:9202:FetchPort:9201:ReplicatePort:9203]
```
Worker log:
```
java.io.IOException: Wait pending actions timeout.
at org.apache.celeborn.service.deploy.worker.storage.TierWriterBase.waitOnNoPending(TierWriter.scala:158)
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Closes#3403 from turboFei/commit_failed.
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 support for MapperEnd/Response in CppClient.
### Why are the changes needed?
To support writing to Celeborn with CppClient.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
By compilation and UTs.
Closes#3372 from HolyLow/issue/celeborn-2070-support-registershuffle-mapperend.
Authored-by: HolyLow <jiaming.xie7@gmail.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
`S3FlushTask` and `OssFlushTask` should close `ByteArrayInputStream` to avoid resource leak.
### Why are the changes needed?
`S3FlushTask` and `OssFlushTask` don't close `ByteArrayInputStream` at present, which may cause resource leak.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3395 from SteNicholas/CELEBORN-2086.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Document introduced metrics into `monitoring.md` including `FetchChunkTransferTime`, `FetchChunkTransferSize`, `FlushWorkingQueueSize`, `LocalFlushCount`, `LocalFlushSize`, `HdfsFlushCount`, `HdfsFlushSize`, `OssFlushCount`, `OssFlushSize`, `S3FlushCount`, `S3FlushSize`.
### Why are the changes needed?
Introduced metrics `FetchChunkTransferTime`, `FetchChunkTransferSize`, `FlushWorkingQueueSize`, `LocalFlushCount`, `LocalFlushSize`, `HdfsFlushCount`, `HdfsFlushSize`, `OssFlushCount`, `OssFlushSize`, `S3FlushCount`, `S3FlushSize` don't document in `monitoring.md`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
No.
Closes#3398 from SteNicholas/document-monitoring.
Authored-by: SteNicholas <programgeek@163.com>
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#3396 from SteNicholas/CELEBORN-894.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Refactoring similar code about configuration usage.
### Why are the changes needed?
Improve scalability for possible new remote storage in the future.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit tests.
Closes#3353 from Kalvin2077/draft.
Authored-by: Kalvin2077 <wk.huang2077@outlook.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
For WorkerStatusTracker, log error for recordWorkerFailure to separate with status change from application heartbeat response.
### Why are the changes needed?
Currently, in `WorkerStatusTracker`, it logs warning for two cases:
1. status change from application heartbeat response
ae40222351/client/src/main/scala/org/apache/celeborn/client/WorkerStatusTracker.scala (L213-L214)
2. `recordWorkerFailure ` on some failures, likes `connectFailedWorkers`.
In our use case, the celeborn cluster is very large and the worker status change frequently, so the log for case 1 is very noisy.
I think that:
1. for case2, it is more critical, should use error level
2. for case1, it might be normal for large celeborn cluster, warning level is fine.
With separated log levels, we can mute the noisy status change from application heartbeat response by setting the log level for `WorkerStatusTracker` to error.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Code review.
Closes#3392 from turboFei/log_level_worker_status.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Support `MapPartitionData` on DFS.
### Why are the changes needed?
`MapPartitionData` only supports on local, which does not support on DFS. It's recommended to support `MapPartitionData` on DFS for MapPartition mode to align the ability of ReducePartition mode.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`WordCountTestWithHDFS`.
Closes#3349 from SteNicholas/CELEBORN-2047.
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?
When workers with higher workloads are excluded, the master does not have a clear log.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#3391 from cxzl25/CELEBORN-2082.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
`id-epoch` cannot locate which application is specific. We can add shuffle key to the log.
```
25/07/25 04:23:06,439 ERROR [celeborn-push-timeout-checker-0] PushDataHandler: PushMergedData replicate failed for partitionLocation: PartitionLocation[
id-epoch:1623-0
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
GA
Closes#3390 from cxzl25/CELEBORN-2081.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Add netty pinned memory metrics
### Why are the changes needed?
We can know more accurately the memory actually allocated from PoolArena.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing uts.
Closes#3019 from leixm/CELEBORN-1793.
Lead-authored-by: Wang, Fei <fwang12@ebay.com>
Co-authored-by: Xianming Lei <31424839+leixm@users.noreply.github.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Added a configuration for client to read non shuffle partition waiting time
### Why are the changes needed?
When the shuffle data of a task is relatively small and there are many empty shuffle partitions, it will take a lot of time for invalid waiting here
### Does this PR introduce _any_ user-facing change?
add configurable
### How was this patch tested?
production environment validation
Closes#3358 from dh20/celeborn_add-20250707.
Lead-authored-by: duanhao-jk <duanhao-jk@360shuke.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?
add code to register gauge to metricRegistry in addGauge method
### Why are the changes needed?
Because one implementation of addGauge does not register the gauge to the metricRegistry, some Sink implementation classes cannot export these gauge metrics. For example, you cannot see JVMCPUTime in the metric file printed by CsvSink, because CsvSink only prints metrics from the registry variable in MetricsSystem.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
cluster test
Closes#3369 from TheodoreLx/fix-gauge-missing.
Lead-authored-by: TheodoreLx <1548069580@qq.com>
Co-authored-by: TheodoreLx <1548069580@qq.com >
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Bump Flink from 1.19.2, 1.20.1 to 1.19.3, 1.20.2.
### Why are the changes needed?
Flink has released v1.19.3 and v1.20.2, which release notes refer to:
- [Apache Flink 1.19.3 Release Announcement](https://flink.apache.org/2025/07/10/apache-flink-1.19.3-release-announcement/)
- [Apache Flink 1.20.2 Release Announcement](https://flink.apache.org/2025/07/10/apache-flink-1.20.2-release-announcement/)
Flink v1.19.3 adds the `getConsumedPartitionType()` interface into `IndexedInputGate`, which refers to https://github.com/apache/flink/pull/26548.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3385 from SteNicholas/CELEBORN-2080.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Fix the metrics units.
1. bytes -> decbytes, see https://github.com/apache/celeborn/pull/2896
```
metrics_FetchChunkTransferSize_Max
metrics_FetchChunkTransferSize_Mean
```
2. bytes -> none, followup https://github.com/apache/celeborn/pull/3362
```
metrics_LocalFlushSize_Count
metrics_HdfsFlushSize_Count
metrics_OssFlushSize_Count
metrics_S3FlushSize_Count
```
3. ms -> ns, followup https://github.com/apache/celeborn/pull/2990
```
metrics_RpcQueueTime_Max
metrics_RpcQueueTime_Mean
metrics_RpcProcessTime_Max
metrics_RpcProcessTime_Mean
```
4. add unit `decbytes` for `metrics_SortedFileSize_Value`, which was not set before
### Why are the changes needed?
Fix the metrics units.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Code review.
Closes#3381 from turboFei/fix_rpc_unit.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Improve `toString` by JEP-280 instead of `ToStringBuilder`.
### Why are the changes needed?
Since Java 9, String Concatenation has been handled better by default.
ID | DESCRIPTION
-- | --
JEP-280 | [Indify String Concatenation](https://openjdk.org/jeps/280)
Backport https://github.com/apache/spark/pull/51572.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3380 from SteNicholas/CELEBORN-2077.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
add data size limitation to inflight data by introducing a new configuration: `celeborn.client.push.maxBytesInFlight.perWorker/total` and defaults to `celeborn.client.push.buffer.max.size * celeborn.client.push.maxReqsInFlight.perWorker/total`.
for backward compatibility, also add a control: `celeborn.client.push.maxReqsInFlight.enabled`.
### Why are the changes needed?
celeborn do supports limiting the number of push inflight requests via `celeborn.client.push.maxReqsInFlight.perWorker/total`. this is a good constraint to memory usage where most requests do not exceed `celeborn.client.push.buffer.max.size`. however, in a vectorized shuffle (like blaze and gluten), a request might be greatly larger then the max buffer size, leading to too much inflight data and results OOM.
### Does this PR introduce _any_ user-facing change?
Yes, add new config for client
### How was this patch tested?
test on local env
Closes#3248 from DDDominik/CELEBORN-1917.
Lead-authored-by: DDDominik <1015545832@qq.com>
Co-authored-by: SteNicholas <programgeek@163.com>
Co-authored-by: DDDominik <zhuangxian@kuaishou.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Fix OpenStreamTime metrics for PbOpenStreamList request
### Why are the changes needed?
For `PbOpenStreamList` request, the `OpenStreamTime` metrics is not calculated.
cf3c05d668/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/FetchHandler.scala (L140-L160)
And the `workerSource.stopTimer(WorkerSource.OPEN_STREAM_TIME, shuffleKey) ` is meaningless inside `handleReduceOpenStreamInternal`. cf3c05d668/worker/src/main/scala/org/apache/celeborn/service/deploy/worker/FetchHandler.scala (L335-L341)
The `handleReduceOpenStreamInternal` is called in
1. inside `handleOpenStreamInternal`, the `OpenStreamTime` metrics is handled correctly
2. for `PbOpenStreamList, the `OpenStreamTime` metrics is not handled.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually testing.
Closes#3376 from turboFei/buf_size.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: SteNicholas <programgeek@163.com>
Added metrics for the amount of data written to different storage types, including Local, HDFS, OSS, and S3
Currently, there is a lack of data volume written to each storage, and it is impossible to monitor the size and speed of writing.
no
Cluster Test
Closes#3362 from TheodoreLx/add-flush-count-metric.
Authored-by: TheodoreLx <1548069580@qq.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
As title.
### Why are the changes needed?
To prevent the dashboard crash for large celeborn cluster.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually testing.
Closes#3373 from turboFei/metrics_instance.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Fix a typo `buffer` to `body` in `ChunkFetchSuccess.toString`.
### Why are the changes needed?
Since the field name of `AbstractMessage` is body, all the other places show `body=...` instead of `buffer=...`. We had better fix this typo for consistency.
Backport https://github.com/apache/spark/pull/51570.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3378 from SteNicholas/chunk-fetch-success.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
`TransportClientFactory` should close channel explicitly to avoid resource leak for timeout or failure.
### Why are the changes needed?
There is resource leak risk for timeout or failure in `TransportClientFactory#internalCreateClient`.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#3368 from SteNicholas/CELEBORN-2068.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Clean up deprecated Guava API usage.
### Why are the changes needed?
There are deprecated Guava API usage, including:
1. Made modifications to Throwables.propagate with reference to https://github.com/google/guava/wiki/Why-we-deprecated-Throwables.propagate
- For cases where it is known to be a checked exception, including `IOException`, `GeneralSecurityException`, `SaslException`, and `RocksDBException`, none of which are subclasses of RuntimeException or Error, directly replaced Throwables.propagate(e) with `throw new RuntimeException(e);`.
- For cases where it cannot be determined whether it is a checked exception or an unchecked exception or Error, use
```
throwIfUnchecked(e);
throw new RuntimeException(e);
```
to replace `Throwables.propagate(e)`.
0c33dd12b1/guava/src/com/google/common/base/Throwables.java (L199-L235)
```
/**
* ...
* deprecated To preserve behavior, use {code throw e} or {code throw new RuntimeException(e)}
* directly, or use a combination of {link #throwIfUnchecked} and {code throw new
* RuntimeException(e)}. But consider whether users would be better off if your API threw a
* different type of exception. For background on the deprecation, read <a
* href="https://goo.gl/Ivn2kc">Why we deprecated {code Throwables.propagate}</a>.
*/
CanIgnoreReturnValue
J2ktIncompatible
GwtIncompatible
Deprecated
public static RuntimeException propagate(Throwable throwable) {
throwIfUnchecked(throwable);
throw new RuntimeException(throwable);
}
```
Backport https://github.com/apache/spark/pull/48248
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI.
Closes#3367 from SteNicholas/CELEBORN-2067.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Bump ap-loader version from 3.0-9 to 4.0-10.
### Why are the changes needed?
`ap-loader` has already released v4.0-10, which release note refers to [Loader for 4.0 (v10): Heatmaps and Native memory profiling](https://github.com/jvm-profiling-tools/ap-loader/releases/tag/4.0-10). It should bump version from 3.0-9 to 4.0-10 for `JVMProfiler`.
Backport https://github.com/apache/spark/pull/51257.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#3359 from SteNicholas/CELEBORN-2057.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Try to use memory storage first if it is available.
To increase performance, if a cluster is set to use MEMORY and HDFS.
### Why are the changes needed?
To keep the old behavior as in release 0.5, always try to use memory storage first because the slots allocator won't allocate slots on memory storage.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA.
Closes#3352 from FMX/b1844-2.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Inspired by [FLINK-38038](https://issues.apache.org/jira/projects/FLINK/issues/FLINK-38038?filter=allissues]), I used [Tongyi Lingma](https://lingma.aliyun.com/) and qwen3-thinking LLM to identify and fix some typo issues in the Celeborn codebase. For example:
- backLog → backlog
- won`t → won't
- can to be read → can be read
- mapDataPartition → mapPartitionData
- UserDefinePasswordAuthenticationProviderImpl → UserDefinedPasswordAuthenticationProviderImpl
### Why are the changes needed?
Remove typos to improve source code readability for users and ease development for developers.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Code and documentation cleanup does not require additional testing.
Closes#3356 from codenohup/fix-typo.
Authored-by: codenohup <huangxu.walker@gmail.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
In the Flink shuffle service plugin, if _RemoteBufferStreamReader_ receives an _EndOfPartition_ event, it will close itself.
At the same time, when the Celeborn worker releases the corresponding stream, it also sends a _BufferStreamEnd_ message to _RemoteBufferStreamReader_, which leads the _ReadClientHandler_ to receive data from an already-closed stream and consequently logs an unexpected warning.

### Why are the changes needed?
Incorrect logs can easily cause confusion for users.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Covered by existing test cases.
Closes#3357 from codenohup/fix-bufferstreamend.
Authored-by: codenohup <huangxu.walker@gmail.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
### What changes were proposed in this pull request?
Add the release 0.6.0 contributor name mapping
### Why are the changes needed?
As title.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Closes#3354 from turboFei/fix_author.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Support write map partition to DFS.
### Why are the changes needed?
`DiskFileInfo` is wrong when flink write,it did not distinguish the types of FileMeta.
indexBuffer will also fail when flink write in the dfs case,because indexBuffer.array will throw `UnsupportedOperationException`(it is allocated by direct memory).
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#3350 from Alibaba-HZY/fix-flink-writer-hdfs.
Lead-authored-by: daowu.hzy <daowu.hzy@alibaba-inc.com>
Co-authored-by: Nicholas Jiang <programgeek@163.com>
Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
…sProcessed
### What changes were proposed in this pull request?
In SkewHandlingWithoutMapRangeValidator, we should put "updateCommitMetadata" ahead of "subPartitionsProcessed.put(endMapIndex, actualCommitMetadata)"
### Why are the changes needed?
In test,we found some cases.When a job enable intergrity checked && enable celeborn.client.adaptive.optimizeSkewedPartitionRead.enabled && job is skew, the last two reduce tasks excutes concurrently may cause validate failed.
task1-attempt1
```
org.apache.celeborn.common.exception.CelebornIOException: AQE Partition 121 failed validation checkwhile processing range startMapIndex: 5 endMapIndex: 1ExpectedCommitMetadata CommitMetadata{bytes=27755354976, crc=CelebornCRC32{current=1320432810}}, ActualCommitMetadata CommitMetadata{bytes=41366743972, crc=CelebornCRC32{current=477185228}},
```
task1-attempt2
```
org.apache.celeborn.common.exception.CelebornIOException: AQE Partition 121 failed validation checkwhile processing range startMapIndex: 5 endMapIndex: 1ExpectedCommitMetadata CommitMetadata{bytes=27755354976, crc=CelebornCRC32{current=1320432810}}, ActualCommitMetadata CommitMetadata{bytes=48072750200, crc=CelebornCRC32{current=755010}},
```
task2-attempt1
```
org.apache.celeborn.common.exception.CelebornIOException: AQE Partition 121 failed validation checkwhile processing range startMapIndex: 5 endMapIndex: 0ExpectedCommitMetadata CommitMetadata{bytes=27755354976, crc=CelebornCRC32{current=1320432810}}, ActualCommitMetadata CommitMetadata{bytes=34660737744, crc=CelebornCRC32{current=953615190}},
```
task2-attempt2
```
org.apache.celeborn.common.exception.CelebornIOException: AQE Partition 121 failed validation checkwhile processing range startMapIndex: 5 endMapIndex: 0ExpectedCommitMetadata CommitMetadata{bytes=27755354976, crc=CelebornCRC32{current=1320432810}}, ActualCommitMetadata CommitMetadata{bytes=54978132968, crc=CelebornCRC32{current=-366062354}},
```
they both read skew Partition 121.And they are the last two reduce tasks of the stage.When task1 executes 'CommitMetadata.checkCommitMetadata(expectedCommitMetadata, currentCommitMetadata)' meanwhile task2 puts its endMapIndex to subPartitionsProcessed but not update commitMeta yet, task1 fails to validate.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Test by skew jobs with optimizeSkewedPartitionRead.enabled.
Closes#3351 from buska88/feature/894.
Authored-by: lijianfu03 <lijianfu@meituan.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
Design doc - https://docs.google.com/document/d/1YqK0kua-5rMufJw57kEIrHHGbLnAF9iXM5GdDweMzzg/edit?tab=t.0#heading=h.n5ldma432qnd
- End to End integrity checks provide additional confidence that Celeborn is producing complete as well as correct data
- The checks are hidden behind a client side config that is false by default. Provides users optionality to enable these if required on a per app basis
- Only compatible with Spark at the moment
- No support for Flink (can be considered in future)
- No support for Columnar Shuffle (can be considered in future)
Writer
- Whenever a mapper completes, it reports crc32 and bytes written on a per partition basis to the driver
Driver
- Driver aggregates the mapper reports - and computes aggregated CRC32 and bytes written on per partitionID basis
Reader
- Each CelebornInputStream will report (int shuffleId, int partitionId, int startMapIndex, int endMapIndex, int crc32, long bytes) to driver when it finished reading all data on the stream
- On every report
- Driver will aggregate the CRC32 and bytesRead for the partitionID
- Driver will aggregate mapRange to determine when all sub-paritions have been read for partitionID have been read
- It will then compare the aggregated CRC32 and bytes read with the expected CRC32 and bytes written for the partition
- There is special handling for skewhandlingwithoutMapRangeSplit scenario as well
- In this case, we report the number of sub-partitions and index of the sub-partition instead of startMapIndex and endMapIndex
There is separate handling for skew handling with and without map range split
As a follow up, I will do another PR that will harden up the checks and perform additional checks to add book keeping that every CelebornInputStream makes the required checks
### Why are the changes needed?
https://issues.apache.org/jira/browse/CELEBORN-894
Note: I am putting up this PR even though some tests are failing, since I want to get some early feedback on the code changes.
### Does this PR introduce _any_ user-facing change?
Not sure how to answer this. A new client side config is available to enable the checks if required
### How was this patch tested?
Unit tests + Integration tests
Closes#3261 from gauravkm/gaurav/e2e_checks_v3.
Lead-authored-by: Gaurav Mittal <gaurav@stripe.com>
Co-authored-by: Gaurav Mittal <gauravkm@gmail.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. Add a new sink and allow the user to store metrics to files.
2. Celeborn will scrape its metrics periodically to make sure that the metric data won't be too large to cause OOM.
### Why are the changes needed?
A long-running worker ran out of memory and found out that the metrics are huge in the heap dump.
As you can see below, the biggest object is the time metric queue, and I got 1.6 million records.
<img width="1516" alt="Screenshot 2025-06-24 at 09 59 30" src="https://github.com/user-attachments/assets/691c7bc2-b974-4cc0-8d5a-bf626ab903c0" />
<img width="1239" alt="Screenshot 2025-06-24 at 14 45 10" src="https://github.com/user-attachments/assets/ebdf5a4d-c941-4f1e-911f-647aa156b37a" />
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
Cluster.
Closes#3346 from FMX/b2045.
Lead-authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Co-authored-by: Ethan Feng <ethanfeng@apache.org>
Co-authored-by: Fei Wang <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
As title.
### Why are the changes needed?
An NPE will be thrown when the ```TransportMessage``` payload is null, and there is no check here.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing ut.
Closes#3330 from Jraaay/main.
Authored-by: Jray <1075860716@qq.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
### What changes were proposed in this pull request?
Specify `extractionDir` of `AsyncProfilerLoader` with `celeborn.worker.jvmProfiler.localDir`.
### Why are the changes needed?
`AsyncProfilerLoader` uses `user.home` directory to store the extracted libraries by default . When `user.home` directory is not initialized, it will cause `AsyncProfilerLoader#load` to fail. `extractionDir` of `AsyncProfilerLoader` could be specified with `celeborn.worker.jvmProfiler.localDir` to avoid failure of loading.
Backport https://github.com/apache/spark/pull/51229.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#3345 from SteNicholas/CELEBORN-2046.
Lead-authored-by: SteNicholas <programgeek@163.com>
Co-authored-by: 子懿 <ziyi.jxf@antgroup.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Proactively cleanup from ChunkStreamManager when stream is closed.
### Why are the changes needed?
Stream gets closed only when shuffle expires at master, which can take a while.
In meantime, workers incur have nontrivial memory utilization.
### Does this PR introduce _any_ user-facing change?
No. Reduces memory usage in workers.
### How was this patch tested?
Existing unit tests
Closes#3343 from mridulm/proactively-cleanup-streams.
Authored-by: Mridul Muralidharan <mridulatgmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
Return SoftSplit status when there is no hard split for pushMergeData
### Why are the changes needed?
HardSplit support was introduced in [CELEBORN-1721](https://issues.apache.org/jira/browse/CELEBORN-1721), and it works well when the client and server are of the same version. However, using a 0.5.x client with a 0.6.x server can significantly impact shuffle write performance. This is because the 0.6.x server returns hardsplit status whenever there is any soft or hard split, leading the 0.5.x client to perform hardsplit on every partition. To maintain compatibility during upgrades, it's essential to preserve the original behavior for 0.5.x clients.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass GA & 1T tpcds with 0.5.4 version client, according to the test results, after applying this PR, revive decreased significantly, and performance improved.
#### test use 0.6.0-rc2 server, 0.5.4 client

#### test use 0.7.0 server + this pr, 0.5.4 client

Closes#3342 from RexXiong/CELEBORN-1721-FOLLOWUP.
Authored-by: Shuang <lvshuang.xjs@alibaba-inc.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
If the create file order list doesn't have the corresponding storage types, use 0 as the index.
### Why are the changes needed?
This is needed in two scenarios:
1. For existing clients, the change partition will select MEMORY as the default storage tier, which will cause the revived partition to utilize memory storage even if the application is not configured to do so. This is the cause of [CELEBORN-2043], and fixed by [CELEBORN-2021](https://github.com/apache/celeborn/pull/3302).
2. Evicted files will need to exclude itself storage type in the create file order list. which means that the storage type does not exist in the create file order list.
### Does this PR introduce _any_ user-facing change?
NO.
### How was this patch tested?
GA and cluster.
Closes#3344 from FMX/b2043.
Authored-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
In our production environment, when obtaining GetReducerFileGroupResponse via broadcast[CELEBORN-1921], failures may occur due to reasons such as:Executor preemption or local disk errors when task writing broadcast data. These scenarios throw a CelebornIOException, which is eventually converted to a FetchFailedException.
However, I think these errors are not caused by shuffle-related metadata loss, so a FetchFailedException should not be thrown to trigger a stage retry. Instead, the task should simply fail and be retried at the task level.
### Why are the changes needed?
To reduce false positive fetch failures.
case 1: During deserialization of the GetReducerFileGroupResponse broadcast, ExecutorLostFailure happend because Container was preempted, leads to reporting a fetch failure.
```
25/06/16 08:39:21 INFO Executor task launch worker for task 30724 SparkUtils: Deserializing GetReducerFileGroupResponse broadcast for shuffle: 0
25/06/16 08:39:21 INFO Executor task launch worker for task 30724 TorrentBroadcast: Started reading broadcast variable 7 with 3 pieces (estimated total size 12.0 MiB)
......
25/06/16 08:39:21 ERROR Executor task launch worker for task 30724 SparkUtils: Failed to deserialize GetReducerFileGroupResponse for shuffle: 0
java.io.IOException: org.apache.spark.SparkException: Exception thrown in awaitResult:
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1387)
at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)
at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)
at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
at org.apache.spark.shuffle.celeborn.SparkUtils.lambda$deserializeGetReducerFileGroupResponse$4(SparkUtils.java:600)
at org.apache.celeborn.common.util.KeyLock.withLock(KeyLock.scala:65)
at org.apache.spark.shuffle.celeborn.SparkUtils.deserializeGetReducerFileGroupResponse(SparkUtils.java:585)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader$$anon$5.apply(CelebornShuffleReader.scala:485)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader$$anon$5.apply(CelebornShuffleReader.scala:480)
at org.apache.celeborn.client.ShuffleClient.deserializeReducerFileGroupResponse(ShuffleClient.java:321)
at org.apache.celeborn.client.ShuffleClientImpl.loadFileGroupInternal(ShuffleClientImpl.java:1876)
at org.apache.celeborn.client.ShuffleClientImpl.lambda$updateFileGroup$9(ShuffleClientImpl.java:1935)
at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853)
at org.apache.celeborn.client.ShuffleClientImpl.updateFileGroup(ShuffleClientImpl.java:1931)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader.read(CelebornShuffleReader.scala:119)
at org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:225)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
at org.apache.spark.scheduler.Task.run(Task.scala:130)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:477)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1428)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:480)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.spark.SparkException: Exception thrown in awaitResult:
at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:302)
at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:105)
at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:89)
at org.apache.spark.storage.BlockManagerMaster.getLocationsAndStatus(BlockManagerMaster.scala:93)
at org.apache.spark.storage.BlockManager.getRemoteBlock(BlockManager.scala:1179)
at org.apache.spark.storage.BlockManager.getRemoteBytes(BlockManager.scala:1341)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBlocks$1(TorrentBroadcast.scala:180)
at scala.runtime.java8.JFunction1$mcVI$sp.apply(JFunction1$mcVI$sp.java:23)
at scala.collection.immutable.List.foreach(List.scala:392)
at org.apache.spark.broadcast.TorrentBroadcast.readBlocks(TorrentBroadcast.scala:169)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$4(TorrentBroadcast.scala:253)
at scala.Option.getOrElse(Option.scala:189)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$2(TorrentBroadcast.scala:231)
at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$1(TorrentBroadcast.scala:226)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1380)
... 40 more
Caused by: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask739bc42 rejected from java.util.concurrent.ScheduledThreadPoolExecutor66c2c5b0[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 0]
at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2047)
at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326)
at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533)
at org.apache.spark.rpc.netty.NettyRpcEnv.askAbortable(NettyRpcEnv.scala:264)
at org.apache.spark.rpc.netty.NettyRpcEndpointRef.askAbortable(NettyRpcEnv.scala:552)
at org.apache.spark.rpc.netty.NettyRpcEndpointRef.ask(NettyRpcEnv.scala:556)
at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:104)
... 54 more
25/06/16 08:39:21 ERROR Executor task launch worker for task 30723 ShuffleClientImpl: Exception raised while call GetReducerFileGroup for 0.
org.apache.celeborn.common.exception.CelebornIOException: Failed to get GetReducerFileGroupResponse broadcast for shuffle: 0
at org.apache.celeborn.client.ShuffleClientImpl.loadFileGroupInternal(ShuffleClientImpl.java:1878)
......
25/06/16 08:39:21 WARN Executor task launch worker for task 30724 CelebornShuffleReader: Handle fetch exceptions for 0-0
org.apache.celeborn.common.exception.CelebornIOException: Failed to load file group of shuffle 0 partition 4643! Failed to get GetReducerFileGroupResponse broadcast for shuffle: 0
at org.apache.celeborn.client.ShuffleClientImpl.updateFileGroup(ShuffleClientImpl.java:1943)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader.read(CelebornShuffleReader.scala:119)
......
Caused by: org.apache.celeborn.common.exception.CelebornIOException: Failed to get GetReducerFileGroupResponse broadcast for shuffle: 0
at org.apache.celeborn.client.ShuffleClientImpl.loadFileGroupInternal(ShuffleClientImpl.java:1878)
at org.apache.celeborn.client.ShuffleClientImpl.lambda$updateFileGroup$9(ShuffleClientImpl.java:1935)
at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853)
at org.apache.celeborn.client.ShuffleClientImpl.updateFileGroup(ShuffleClientImpl.java:1931)
... 27 more
```
case 2: During deserialization of the GetReducerFileGroupResponse broadcast, a failure to create the local directory leads to reporting a fetch failure.
```
25/05/27 07:27:03 INFO Executor task launch worker for task 20399 SparkUtils: Deserializing GetReducerFileGroupResponse broadcast for shuffle: 1
25/05/27 07:27:03 INFO Executor task launch worker for task 20399 TorrentBroadcast: Started reading broadcast variable 5 with 1 pieces (estimated total size 4.0 MiB)
25/05/27 07:27:03 INFO Executor task launch worker for task 20399 TorrentBroadcast: Reading broadcast variable 5 took 0 ms
25/05/27 07:27:03 INFO Executor task launch worker for task 20399 MemoryStore: Block broadcast_5 stored as values in memory (estimated size 980.4 KiB, free 6.3 GiB)
25/05/27 07:27:03 WARN Executor task launch worker for task 20399 BlockManager: Putting block broadcast_5 failed due to exception java.io.IOException: Failed to create local dir in /data12/hadoop/yarn/nm-local-dir/usercache/......
25/05/27 07:27:03 WARN Executor task launch worker for task 20399 BlockManager: Block broadcast_5 was not removed normally.
25/05/27 07:27:03 ERROR Executor task launch worker for task 20399 Utils: Exception encountered
java.io.IOException: Failed to create local dir in /data12/hadoop/yarn/nm-local-dir/usercache/......
at org.apache.spark.storage.DiskBlockManager.getFile(DiskBlockManager.scala:93)
at org.apache.spark.storage.DiskStore.remove(DiskStore.scala:114)
at org.apache.spark.storage.BlockManager.removeBlockInternal(BlockManager.scala:2050)
at org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1574)
at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1611)
at org.apache.spark.storage.BlockManager.putIterator(BlockManager.scala:1467)
at org.apache.spark.storage.BlockManager.putSingle(BlockManager.scala:1936)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$4(TorrentBroadcast.scala:262)
at scala.Option.getOrElse(Option.scala:189)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$2(TorrentBroadcast.scala:231)
at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
at org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$1(TorrentBroadcast.scala:226)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1380)
at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)
at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)
at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
at org.apache.spark.shuffle.celeborn.SparkUtils.lambda$deserializeGetReducerFileGroupResponse$4(SparkUtils.java:600)
at org.apache.celeborn.common.util.KeyLock.withLock(KeyLock.scala:65)
at org.apache.spark.shuffle.celeborn.SparkUtils.deserializeGetReducerFileGroupResponse(SparkUtils.java:585)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader$$anon$5.apply(CelebornShuffleReader.scala:485)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader$$anon$5.apply(CelebornShuffleReader.scala:480)
at org.apache.celeborn.client.ShuffleClient.deserializeReducerFileGroupResponse(ShuffleClient.java:321)
at org.apache.celeborn.client.ShuffleClientImpl.loadFileGroupInternal(ShuffleClientImpl.java:1876)
at org.apache.celeborn.client.ShuffleClientImpl.lambda$updateFileGroup$9(ShuffleClientImpl.java:1935)
at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1877)
at org.apache.celeborn.client.ShuffleClientImpl.updateFileGroup(ShuffleClientImpl.java:1931)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader.read(CelebornShuffleReader.scala:119)
at org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:225)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:60)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:102)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
at org.apache.spark.scheduler.Task.run(Task.scala:130)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:477)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1428)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:480)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
25/05/27 07:27:03 ERROR Executor task launch worker for task 20399 ShuffleClientImpl: Exception raised while call GetReducerFileGroup for 1.
org.apache.celeborn.common.exception.CelebornIOException: Failed to get GetReducerFileGroupResponse broadcast for shuffle: 1
......
25/05/27 07:27:03 WARN Executor task launch worker for task 20399 CelebornShuffleReader: Handle fetch exceptions for 1-0
org.apache.celeborn.common.exception.CelebornIOException: Failed to load file group of shuffle 1 partition 4001! Failed to get GetReducerFileGroupResponse broadcast for shuffle: 1
at org.apache.celeborn.client.ShuffleClientImpl.updateFileGroup(ShuffleClientImpl.java:1943)
at org.apache.spark.shuffle.celeborn.CelebornShuffleReader.read(CelebornShuffleReader.scala:119)
......
Caused by: org.apache.celeborn.common.exception.CelebornIOException: Failed to get GetReducerFileGroupResponse broadcast for shuffle: 1
at org.apache.celeborn.client.ShuffleClientImpl.loadFileGroupInternal(ShuffleClientImpl.java:1878)
......
```
### Does this PR introduce _any_ user-facing change?
When `ShuffleClient.deserializeReducerFileGroupResponse(shuffleId, response.broadcast())` return null, will not report a fetch failure, instead, the task will simply fail.
### How was this patch tested?
Long-running Production Validation
Closes#3341 from vastian180/CELEBORN-2040.
Lead-authored-by: caohaotian <caohaotian@meituan.com>
Co-authored-by: Fei Wang <cn.feiwang@gmail.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Support testing of dynamic configuration management cli.
### Why are the changes needed?
The tests of dynamic configuration management cli are disabled since dynamic conf is not enabled in unit tests, which should support testing dynamic configuration management cli.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
`TestCelebornCliCommands`.
Closes#3340 from SteNicholas/CELEBORN-1056.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Fixes the FetchFailure handling logic in shouldReportShuffleFetchFailure method to properly handle cases where TaskSetManager cannot be found for a given task ID.
### Why are the changes needed?
The current implementation incorrectly reports FetchFailure when TaskSetManager is not found, which leads to false positive failures in normal fault tolerance scenarios. This happens because:
1. Executor Lost scenarios: When executors are lost due to resource preemption or failures, the associated TaskSetManager gets cleaned up, making it unavailable for lookup
2. Stage cancellation: Cancelled or completed stages may have their TaskSetManager removed
These are all normal scenarios in Spark's fault tolerance mechanism and should not be treated as shuffle failures. The current behavior can cause unnecessary job failures and confusion in debugging actual shuffle issues.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
UT, Long-running Production Validation
Closes#3339 from gaoyajun02/CELEBORN-2042.
Authored-by: gaoyajun02 <gaoyajun02@meituan.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Batching few log line on celeborn client side, which logs too much data in spark driver logs for large application.
- Change partition print updated partition for each worker individually and prints too many lines if many workers are involved in shuffle.
```
25/06/02 08:04:29 INFO LifecycleManager: Reserve buffer success for shuffleId 6
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2477 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 1285 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2660 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2194 epoch from 0 to 1), (partition 1760 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 1429 epoch from 0 to 1), (partition 2300 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 517 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2627 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2901 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2903 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2067 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 569 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2633 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2813 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 1817 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 148 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 2098 epoch from 0 to 1)].
25/06/02 08:04:29 INFO ChangePartitionManager: [Update partition] success for shuffle 6, succeed partitions: [(partition 1554 epoch from 0 to 1)].
```
- Clear shuffle print each shuffle id individually
```
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 0.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 1.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 2.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 3.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 4.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 5.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 6.
25/06/02 08:01:51 INFO LifecycleManager: Clear shuffle 7.
```
### Why are the changes needed?
Both of the above logs gets printed a lot on celeborn client and can be merged in a single line.
### Does this PR introduce _any_ user-facing change?
Client logs will have slight change.
### How was this patch tested?
NA
Closes#3313 from s0nskar/improve_logs.
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?
Support upsert and delete of dynamic configuration management.
### Why are the changes needed?
There is only listing dynamic configuration interface for dynamic configuration management. It should support upserting and deleting dynamic configuration.
### Does this PR introduce _any_ user-facing change?
- Rest API:
- `/api/v1/conf/dynamic/upsert` to upsert dynamic configurations.
- `/api/v1/conf/dynamic/delete` to delete dynamic configurations.
- CLI:
- `--upsert-dynamic-conf` to upsert dynamic configurations.
- `--delete-dynamic-conf` to upsert dynamic configurations.
### How was this patch tested?
- `ConfigServiceSuiteJ`
- `ApiV1BaseResourceSuite`
- `TestCelebornCliCommands`
Closes#3323 from SteNicholas/CELEBORN-1056.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
Update the config version to 0.5.5 for `celeborn.worker.flusher.local.gatherAPI.enabled`.
### Why are the changes needed?
Followup https://github.com/apache/celeborn/pull/3335
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
GA
Closes#3338 from turboFei/CELEBORN-1931_follow.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
<!--
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?
Added a commit files request fail count metric.
### Why are the changes needed?
To monitor and tune the configurations around the commit files workflow.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Local setup
<img width="739" alt="Screenshot 2025-06-04 at 10 51 06 AM" src="https://github.com/user-attachments/assets/d6256028-d8b7-4a81-90b1-3dcbf61adeba" />
Closes#3307 from s0nskar/commit_metric.
Authored-by: Sanskar Modi <sanskarmodi97@gmail.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
As title
### Why are the changes needed?
As title
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
PASS GA & grafana
Closes#3333 from RexXiong/CELEBORN-1817-FOLLOWUP.
Authored-by: Shuang <lvshuang.xjs@alibaba-inc.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>
### What changes were proposed in this pull request?
For release mode, check the JAVA_HOME variables and set JDK version to 8 by default
```
./build/make-distribution.sh --sbt-enabled --release
```
### Why are the changes needed?
1. Address comments: https://github.com/apache/celeborn/pull/3282#discussion_r2149146734
2. Set JDK 8 for release mode by default is necessary
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manual test.
Closes#3331 from turboFei/JAVA_HOME.
Authored-by: Wang, Fei <fwang12@ebay.com>
Signed-off-by: Wang, Fei <fwang12@ebay.com>